You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/02/23 12:17:35 UTC

[GitHub] [flink] afedulov opened a new pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

afedulov opened a new pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195
 
 
   ## What is the purpose of the change
   
   [FLINK-11952](https://issues.apache.org/jira/browse/FLINK-11952) introduced Plugins mechanism into Flink. This PR makes it possible to benefit from using this new functionality during the MetricReporters initialization. Instead of placing MetricReporters JARs into`/libs`, it is now additionally possible (and encouraged) to convert them into plugins and use the `/plugins` folder for initialization via independent plugin classloaders.
   
   Note that to enable this functionality, a bounded type of Plugin was removed from the PluginLoader/PluginManager methods. This removal did not affect any existing functionality, including FileSystems initialization. The adjustment was required for the following reason: `MetricReporter` interface, which is located in the `flink-metrics-core` module is currently not a `Plugin`. `Plugin` interface is located in `flink-core` and this module already has a depency on `flink-metrics-core`. This makes in impossible to mark `MetricReporter` as  `Plugin` without introducing a circular dependency. An  alternative approach could be extraction of the whole plugin classloading functionality out of `flink-core` into a separate module.
   
   Another important consideration - current `PluginClassLoader` implementation uses parent-first initialization for classes, which start with Flink-native package prefixes, such as *org.apache.flink.*. Currently, metrics reporters shipped with Flink all have the same package prefix. This leads to the following situation: if there is a version of `PluginMetricFactory` for a particular reporter found in the /lib folder, JARs for this reporter in /plugins directory will effectively be ignored. The opposite priority might actually be desirable to improve user's experience when transitioning to plugins-based metrics initialization approach.
   
   ## Brief change log
   
     - `PluginManager`, already created for `FileSystems` initialization, is propagated from all Flink entrypoints into metrics reporters initializer
     - Reporters initializer (`ReporterSetup`) combines initialization via existing `ServiceLoader` approach with the new approach of loading plugins
   - `PrometheusReporter` is adjusted to be compatible with the new plugins mechanism. Proper functioning is verified with the end-to-end tests
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
     - Added integration tests for end-to-end tests
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes)
     - If yes, how is the feature documented? (not documented)
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395375550
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -260,16 +262,26 @@ public void submitSQLJob(SQLJobSubmission job) throws IOException {
 	}
 
 	public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException {
-		final Optional<Path> reporterJarOptional;
-		try (Stream<Path> logFiles = Files.walk(opt)) {
-			reporterJarOptional = logFiles
+		copyOptJars(jarNamePrefix, lib);
+	}
+
+	public void copyOptJarsToPlugins(String jarNamePrefix) throws FileNotFoundException, IOException {
 
 Review comment:
   > and modify FlinkDistribution#moveJar to handle this location appropriately.
    could you clarify? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155617305",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155680617",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * eccc41d96524bd8ff6bfa6bfa36f366689255893 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/155617305) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691) 
   * 19b07382fedcab4e7c4866eb260a12cde4780538 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/155680617) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 21c0efa3b9bae9e5bb23338c23483f41f0d1e738 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151445413) 
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395375550
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -260,16 +262,26 @@ public void submitSQLJob(SQLJobSubmission job) throws IOException {
 	}
 
 	public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException {
-		final Optional<Path> reporterJarOptional;
-		try (Stream<Path> logFiles = Files.walk(opt)) {
-			reporterJarOptional = logFiles
+		copyOptJars(jarNamePrefix, lib);
+	}
+
+	public void copyOptJarsToPlugins(String jarNamePrefix) throws FileNotFoundException, IOException {
 
 Review comment:
   > and modify FlinkDistribution#moveJar to handle this location appropriately.
   - could you clarify? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386663744
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395330122
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -210,17 +213,31 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return namedOrderedReporters;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
-
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.info("Prepare reporter factories (from both SPIs and Plugins):");
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.info("Found reporter factory {} at {} ",
+						factoryClassName,
+						new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   No hard feelings. You can safely ignore that comment. Just wanted to point it out and closing with a cite of your link 
   
   > It has been strongly criticized as both ugly in style, and ambiguous in legal documents .
   
   Same in academic writing.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386468691
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
 ##########
 @@ -119,7 +120,7 @@
 
 	private boolean shutdown;
 
-	public TaskManagerRunner(Configuration configuration, ResourceID resourceId) throws Exception {
+	public TaskManagerRunner(Configuration configuration, ResourceID resourceId, PluginManager pluginManager) throws Exception {
 
 Review comment:
   In the PRs current state `pluginManager` should be annotated with `Nullable`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395366353
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
 ##########
 @@ -160,13 +161,14 @@ public void startCluster() throws ClusterEntrypointException {
 		LOG.info("Starting {}.", getClass().getSimpleName());
 
 		try {
-
-			configureFileSystems(configuration);
+			//TODO: push down filesystem initialization into runCluster - initializeServices (?)
 
 Review comment:
   If it is something non-trivial and hard to make a call about, I would propose to skip this refactoring for now.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386465432
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -38,11 +38,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.BufferedReader;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStreamReader;
+import java.io.*;
 
 Review comment:
   we don't allow star imports

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395335333
 
 

 ##########
 File path: flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporterFactory.java
 ##########
 @@ -0,0 +1,34 @@
+/*
+ * 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.metrics.prometheus;
+
+import org.apache.flink.core.plugin.Plugin;
+import org.apache.flink.metrics.reporter.MetricReporterFactory;
+
+import java.util.Properties;
+
+/**
+ * {@link MetricReporterFactory} for {@link PrometheusReporter}.
+ */
+public class PrometheusReporterFactory implements MetricReporterFactory, Plugin {
+
+	@Override
+	public PrometheusReporter createMetricReporter(Properties properties) {
+		return new PrometheusReporter();
 
 Review comment:
   It seems that this will pull a rather large refactoring with it, because of the call to `super.open(config)` in the `open` methods and because of having to reconcile different configuration containers - `Properties` vs `MetricsConfig`. I would prefer to address it in a separate refactoring PR, if possible.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155617305",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155680617",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6770",
       "triggerID" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155925108",
       "triggerID" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * 6a088a5f122ccce53e5baa97603eb8b0a9599247 Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/155925108) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6770) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395618828
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -210,17 +213,31 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return namedOrderedReporters;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
-
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.info("Prepare reporter factories (from both SPIs and Plugins):");
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.info("Found reporter factory {} at {} ",
+						factoryClassName,
+						new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   I think we should stop discussing this here; this is relevant for the entirety of the documentation, and we could get much more experienced people involved if we target that instead.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155617305",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * eccc41d96524bd8ff6bfa6bfa36f366689255893 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/155617305) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395337241
 
 

 ##########
 File path: flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporterFactory.java
 ##########
 @@ -0,0 +1,34 @@
+/*
+ * 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.metrics.prometheus;
+
+import org.apache.flink.core.plugin.Plugin;
+import org.apache.flink.metrics.reporter.MetricReporterFactory;
+
+import java.util.Properties;
+
+/**
+ * {@link MetricReporterFactory} for {@link PrometheusReporter}.
+ */
+public class PrometheusReporterFactory implements MetricReporterFactory, Plugin {
 
 Review comment:
   Added.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394245745
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -210,17 +213,31 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return namedOrderedReporters;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
-
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.info("Prepare reporter factories (from both SPIs and Plugins):");
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.info("Found reporter factory {} at {} ",
+						factoryClassName,
+						new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   It's frequently used in our code and documentation; I don't think we gain anything by watching out for things like this.
   Personally I find it less ambiguous than just `or`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386613981
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 
 Review comment:
   Is this an accepted style in Flink? I mostly see "classic" variant with `} catch ...` (including the same class in loadReporterFactories() method.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386472491
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 
 Review comment:
   Given that we are iterating over all factories anyway we should be able to move this into the while loop.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394183300
 
 

 ##########
 File path: flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporterFactory.java
 ##########
 @@ -0,0 +1,34 @@
+/*
+ * 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.metrics.prometheus;
+
+import org.apache.flink.core.plugin.Plugin;
+import org.apache.flink.metrics.reporter.MetricReporterFactory;
+
+import java.util.Properties;
+
+/**
+ * {@link MetricReporterFactory} for {@link PrometheusReporter}.
+ */
+public class PrometheusReporterFactory implements MetricReporterFactory, Plugin {
 
 Review comment:
   We need a second factory for the `PrometheusPushGatewayReporter`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395365114
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -210,17 +213,31 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return namedOrderedReporters;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
-
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.info("Prepare reporter factories (from both SPIs and Plugins):");
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.info("Found reporter factory {} at {} ",
+						factoryClassName,
+						new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   I thing the problem is that we have too many variants to describe with a single coordinating conjunction like "or" (plus "or" can unfortunately be both inclusive and exclusive). Cases:
   (xx) _ ()
   () _ (xx)
   (x) _ (x)
   (xx) _ (xx)
   
   Alternatives like "a or b or both" also do not work, because in this particular case they can be easily be interpreted as if the "(x) _ (x)" case is not an issue (because of "Multiple" in the beginning). Seems to me like the best case to indicate this ambiguity is is to use "and/or" so that people will be aware of multiple ways this can happen.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r388346247
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   Collectors.toCollection(TreeSet::new) to get rid of the next few lines.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395348055
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   @AHeise I have applied this refactoring but then understood that I probably did not get what you actually propose. Could you clarify? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386644501
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
 ##########
 @@ -119,7 +120,7 @@
 
 	private boolean shutdown;
 
-	public TaskManagerRunner(Configuration configuration, ResourceID resourceId) throws Exception {
+	public TaskManagerRunner(Configuration configuration, ResourceID resourceId, PluginManager pluginManager) throws Exception {
 
 Review comment:
   I am  not sure what should be the expected contract here. ReporterSetup will currently only work with the null pluginManager if no reporters are configured (namedReporters.isEmpty()). We could maybe add a checkNonNull in  the ReporterSetup after the first return due to empty namedReporters.
   
   (TaskManagerRunnerTest is now fixed to initialize the PluginManager properly) 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 21c0efa3b9bae9e5bb23338c23483f41f0d1e738 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151445413) 
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 3aded2be8a0019e7b025d811d204e92d36dd24e7 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393840337
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 		String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, "");
-		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
 
 Review comment:
   Done.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 3aded2be8a0019e7b025d811d204e92d36dd24e7 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/153641242) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155617305",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155680617",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6770",
       "triggerID" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155925108",
       "triggerID" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * 19b07382fedcab4e7c4866eb260a12cde4780538 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/155680617) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699) 
   * 6a088a5f122ccce53e5baa97603eb8b0a9599247 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/155925108) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6770) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395348055
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   @AHeise I have applied this refactoring but then understood that I probably really did not get what you actually propose. Could you please clarify? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395376334
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -260,16 +262,26 @@ public void submitSQLJob(SQLJobSubmission job) throws IOException {
 	}
 
 	public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException {
-		final Optional<Path> reporterJarOptional;
-		try (Stream<Path> logFiles = Files.walk(opt)) {
-			reporterJarOptional = logFiles
+		copyOptJars(jarNamePrefix, lib);
+	}
+
+	public void copyOptJarsToPlugins(String jarNamePrefix) throws FileNotFoundException, IOException {
 
 Review comment:
   I assume you mean to modify FlinkDistribution#mapJarLocationToPath ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r397030268
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   You've mentioned "to get rid of the next few lines", but this Tree data structure is further used as a container that is filled using some conditional logic and returned from the method, it is not just about having the input entries sorted. Do you propose to rewrite it?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395298324
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 
 Review comment:
   @zentol Ok, I find it a bit strange, but I am not setting the rules here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d693668c44effeffafc944835df873e3978f88b9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150196559) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r388344724
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 		String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, "");
-		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
 
 Review comment:
   👍 to split up refactoring from actual commit. But in general also 👍 to refactorings.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386477745
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 
 Review comment:
   same as above

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r396809894
 
 

 ##########
 File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
 ##########
 @@ -120,24 +122,51 @@ public static void checkOS() {
 	public final DownloadCache downloadCache = DownloadCache.get();
 
 	@Test
-	public void testReporter() throws Exception {
-		dist.copyOptJarsToLib("flink-metrics-prometheus");
+	public void reporterWorksWhenFoundInLibsViaReflection() throws Exception {
+		dist.copyOptJarsToLib(PROMETHEUS_JAR_PREFIX);
 
 Review comment:
   @zentol It seems that this updated approach that got merged into master does not support the kinds of tests that we would need to do for the supported scenarios (see reporterWorksWhenFoundInLibsViaReflection, reporterWorksWhenFoundInPluginsViaReflection, reporterWorksWhenFoundBothInPluginsAndLibsViaFactories in this PR). How should we proceed? I can either bend the FlinkResource implementation back to the state where it supports modifications of the underlying resources after creation and keep the initialization in @ Rule (a hack), or reinitizliae FlinkResource in every test (probably a bad idea from performance and overhead perspectives). What do you think?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155617305",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155680617",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * 19b07382fedcab4e7c4866eb260a12cde4780538 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/155680617) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r388348348
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
 
 Review comment:
   Takes a while until we get proper SPI, so I'd add the commented code to increase usability.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * d07ec693eb90e33071e710be0b774fc995f05867 Travis: [CANCELED](https://travis-ci.com/github/flink-ci/flink/builds/154179607) 
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590062848
 
 
   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit d693668c44effeffafc944835df873e3978f88b9 (Sun Feb 23 12:20:08 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394147265
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
 
 Review comment:
   So this would then only work for two plugins. I don't see a way around that without proper SPI. 
   I'd probably remove the commented code completely; it shouldn't be hard to bring back and would avoid some confusion.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol merged pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol merged pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393832500
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 
 Review comment:
   This code was touched anyhow, because of refactoring, so I think it should be OK to do such things, unless you have a strong opinion about this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393580453
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   Strictly speaking it then should be 'and/or', because 'or' could mean that there are other two jars both either in 'lib' or 'plugins'. I think and was more fitting, but I am also ok with and/or

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386630929
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
 
 Review comment:
   ping @pnowojski @AHeise  (depends if we want to proceed with the classloading modifications)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r388313102
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -77,6 +77,7 @@
 	private Path conf;
 
 Review comment:
   Commit message should explain what's actually happening.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393832500
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 
 Review comment:
   This whole code block was already "touched" anyhow, because of refactoring, so I think it should be OK to do such things, unless you have a strong opinion.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386474722
 
 

 ##########
 File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
 ##########
 @@ -185,6 +214,7 @@ public void testReporter() throws Exception {
 
 			checkMetricAvailability(client, "flink_jobmanager_numRegisteredTaskManagers");
 			checkMetricAvailability(client, "flink_taskmanager_Status_Network_TotalMemorySegments");
+
 
 Review comment:
   revert

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393950579
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 		String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, "");
-		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
-			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
-			.collect(Collectors.toSet());
 
-		// use a TreeSet to make the reporter order deterministic, which is useful for testing
-		Set<String> namedReporters = new TreeSet<>(String::compareTo);
-		// scan entire configuration for "metric.reporter" keys and parse individual reporter configurations
-		for (String key : configuration.keySet()) {
-			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
-				Matcher matcher = reporterClassPattern.matcher(key);
-				if (matcher.matches()) {
-					String reporterName = matcher.group(1);
-					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
-						if (namedReporters.contains(reporterName)) {
-							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
-						} else {
-							namedReporters.add(reporterName);
-						}
-					} else {
-						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
-					}
-				}
-			}
-		}
+		Set<String> namedReporters = findEnabledReportersInConfiguration(configuration,
+			includedReportersString);
 
 		if (namedReporters.isEmpty()) {
 			return Collections.emptyList();
 		}
 
-		List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+		final Map<String, MetricReporterFactory> reporterFactories = loadAvailableReporterFactories(pluginManager);
+		LOG.debug("Loaded Reporter Factories: {}", reporterFactories);
+		final List<Tuple2<String, Configuration>> reporterConfigurations = loadReporterConfigurations(configuration, namedReporters);
+		LOG.debug("Loaded Reporter Configurations: {}", reporterConfigurations);
 
 Review comment:
   Addressed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386628387
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -38,11 +38,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.BufferedReader;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStreamReader;
+import java.io.*;
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d693668c44effeffafc944835df873e3978f88b9 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150196559) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * a244cee274bab74683c25f90bfd515698bc04c95 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/153812547) 
   * 7ce9ad87a780aab19b9c97680d7af5f648c13f3b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r388310406
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
 ##########
 @@ -202,9 +204,11 @@ private SecurityContext installSecurityContext(Configuration configuration) thro
 		return SecurityUtils.getInstalledContext();
 	}
 
-	private void runCluster(Configuration configuration) throws Exception {
+	private void runCluster(Configuration configuration, PluginManager pluginManager) throws Exception {
 		synchronized (lock) {
-			initializeServices(configuration);
+
+			//TODO: Ask why FileSystem is not initialized here too.
 
 Review comment:
   Just tagging TODO.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 57db59bd20c14d5a0872f6d35b2b3220624ec96b Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/153672077) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395375550
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -260,16 +262,26 @@ public void submitSQLJob(SQLJobSubmission job) throws IOException {
 	}
 
 	public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException {
-		final Optional<Path> reporterJarOptional;
-		try (Stream<Path> logFiles = Files.walk(opt)) {
-			reporterJarOptional = logFiles
+		copyOptJars(jarNamePrefix, lib);
+	}
+
+	public void copyOptJarsToPlugins(String jarNamePrefix) throws FileNotFoundException, IOException {
 
 Review comment:
   > and modify FlinkDistribution#moveJar to handle this location appropriately.
   
   could you clarify? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394238050
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/metrics/ReporterSetupTest.java
 ##########
 @@ -21,6 +21,7 @@
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.core.plugin.PluginManager;
 
 Review comment:
   unused?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393570847
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
 
 Review comment:
   @AHeise the jar path for existing metrics reporters will currently point to the same file due to parent-first loading of org.apache.flink packages (even if one of the jars is in /plugin directory). It might be misleading, so I wanted to keep the note for improvement for later, for when the loading is done differently.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394144042
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
 ##########
 @@ -160,13 +161,14 @@ public void startCluster() throws ClusterEntrypointException {
 		LOG.info("Starting {}.", getClass().getSimpleName());
 
 		try {
-
-			configureFileSystems(configuration);
+			//TODO: push down filesystem initialization into runCluster - initializeServices (?)
 
 Review comment:
   That's something that @zentol knows much better. It sounds plausible to me. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 1613baf0244adc63bf029415e97227a6770591c9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151434781) 
   * 21c0efa3b9bae9e5bb23338c23483f41f0d1e738 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393839754
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 
 Review comment:
   Ping @AHeise @zentol - could we agree on something here?
   I am always "pro extensive logging", but this could be professional deformation. Being able to "on-demand" see what is going on is very valuable for production systems. We could declare somewhere for Flink in general that if you choose to run with debug log level, some potentially sensitive information could leak into logs. My arguments are:
   1) If someone has uncontrolled access to the log files on your machine in production, content of this file is probably not the biggest of your problems.
   2) Running with debug level is not a "normal" scenario - this is intended for hands on investigation of issues. Log level for potentially compromisable external systems could be explicitly set to trace in such cases.
   3) We have been "leaking" this data in the current versions with info (!) level without much concern

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393842876
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
 
 Review comment:
   Seems like a silent skip . This is old code, maybe @zentol could comment.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395326037
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -210,17 +213,31 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return namedOrderedReporters;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
-
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.info("Prepare reporter factories (from both SPIs and Plugins):");
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.info("Found reporter factory {} at {} ",
+						factoryClassName,
+						new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   https://en.wikipedia.org/wiki/And/or 
   
   `It is used as an inclusive "or" (as in logic and mathematics), while an "or" in spoken language might be inclusive or exclusive.`
   
   Seems to me like something that reduces ambiguity, as @zentol said.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395365114
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -210,17 +213,31 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return namedOrderedReporters;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
-
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.info("Prepare reporter factories (from both SPIs and Plugins):");
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.info("Found reporter factory {} at {} ",
+						factoryClassName,
+						new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   I thing the problem is that we have too many variants to describe with a single coordinating conjunction like "or" (plus "or" can unfortunately be both inclusive and exclusive). Cases:
   (xx) _ ()
   () _ (xx)
   (x) _ (x)
   (xx) _ (xx)
   
   Alternatives like "x or y or both" also do not work, because in this particular case they can be easily be interpreted as if the "(x) _ (x)" case is not an issue (because of "Multiple" in the beginning). Seems to me like the best case to indicate this ambiguity is is to use "and/or" so that people will be aware for what kinds of problem potentially to look for.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386477973
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 
 Review comment:
   revert

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393840952
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 		String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, "");
-		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
 
 Review comment:
   Done.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393848386
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   Thanks, changes as proposed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 21c0efa3b9bae9e5bb23338c23483f41f0d1e738 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151445413) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r388346850
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
 
 Review comment:
   warn if it doesn't match?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394147676
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -210,17 +213,31 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return namedOrderedReporters;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
-
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.info("Prepare reporter factories (from both SPIs and Plugins):");
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.info("Found reporter factory {} at {} ",
+						factoryClassName,
+						new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   `/` means `or`. `and/or` is `and or or`, which can be simplified to `or`. (`or` is not `xor` in English)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r388347812
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   Should be 'or'

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7ce9ad87a780aab19b9c97680d7af5f648c13f3b Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/154170382) 
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155617305",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155680617",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6770",
       "triggerID" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155925108",
       "triggerID" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * 6a088a5f122ccce53e5baa97603eb8b0a9599247 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/155925108) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6770) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395299396
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 		String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, "");
-		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
 
 Review comment:
   As this did not come up in the second round I consider the new split of commits as appropriate. Resolving.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395330122
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -210,17 +213,31 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return namedOrderedReporters;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
-
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.info("Prepare reporter factories (from both SPIs and Plugins):");
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.info("Found reporter factory {} at {} ",
+						factoryClassName,
+						new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   No hard feeling. You can safely ignore that comment. Just wanted to point it out and closing with a cite of your link 
   
   > It has been strongly criticized as both ugly in style, and ambiguous in legal documents .
   
   Same in academic writing.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393848386
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   Thanks, changed as proposed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395661258
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -210,17 +213,31 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return namedOrderedReporters;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
-
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.info("Prepare reporter factories (from both SPIs and Plugins):");
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.info("Found reporter factory {} at {} ",
+						factoryClassName,
+						new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   I leave it as "and/or" here, and then we comb through the docs and sources to address it separately.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r396968257
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   `Set<String> namedOrderedReporters =reporterListPattern.splitAsStream(includedReportersString)
   			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
   			.collect(Collectors.toCollection(TreeSet::new));`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395619382
 
 

 ##########
 File path: flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporterFactory.java
 ##########
 @@ -0,0 +1,34 @@
+/*
+ * 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.metrics.prometheus;
+
+import org.apache.flink.core.plugin.Plugin;
+import org.apache.flink.metrics.reporter.MetricReporterFactory;
+
+import java.util.Properties;
+
+/**
+ * {@link MetricReporterFactory} for {@link PrometheusReporter}.
+ */
+public class PrometheusReporterFactory implements MetricReporterFactory, Plugin {
+
+	@Override
+	public PrometheusReporter createMetricReporter(Properties properties) {
+		return new PrometheusReporter();
 
 Review comment:
   hmm that is true, I suppose the dual nature of the Prometheus reporters make things a bit funky. Let's leave it like this for now.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r388312499
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
 ##########
 @@ -119,7 +120,7 @@
 
 	private boolean shutdown;
 
-	public TaskManagerRunner(Configuration configuration, ResourceID resourceId) throws Exception {
+	public TaskManagerRunner(Configuration configuration, ResourceID resourceId, PluginManager pluginManager) throws Exception {
 
 Review comment:
   `FileSystem` can also take a null for plugin manager. We should probably extract an interface and have a no-op implementation instead. That's out of scope for this PR though.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386604078
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 
 Review comment:
   Good point. I am not 100% sure about it - do we have a Flink-wide way to handle such cases? I guess the cleanest approach would be to have a special set of keys that are considered sensitive, which have to be obfuscated prior to logging. Seeing which other config values are used during the initialization could be generally pretty useful for debugging. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395375423
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -260,16 +262,26 @@ public void submitSQLJob(SQLJobSubmission job) throws IOException {
 	}
 
 	public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException {
-		final Optional<Path> reporterJarOptional;
-		try (Stream<Path> logFiles = Files.walk(opt)) {
-			reporterJarOptional = logFiles
+		copyOptJars(jarNamePrefix, lib);
+	}
+
+	public void copyOptJarsToPlugins(String jarNamePrefix) throws FileNotFoundException, IOException {
 
 Review comment:
   But this is not really "copying jars", right? It will actually move the file from opt to lib or plugins. The problem is that one of the cases I would like to test required the jar to be actually copied to both.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394181288
 
 

 ##########
 File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
 ##########
 @@ -120,24 +122,51 @@ public static void checkOS() {
 	public final DownloadCache downloadCache = DownloadCache.get();
 
 	@Test
-	public void testReporter() throws Exception {
-		dist.copyOptJarsToLib("flink-metrics-prometheus");
+	public void reporterWorksWhenFoundInLibsViaReflection() throws Exception {
+		dist.copyOptJarsToLib(PROMETHEUS_JAR_PREFIX);
 
 Review comment:
   this will also need adjustments after a rebase, as this test now works against the `FlinkResource` interface, where jar copies and configuration settings are done as part of the `FlinkResource` setup.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386613981
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 
 Review comment:
   Is this an accepted style in Flink? I mostly see "classic" variant with `} catch ...` , including the same class in loadReporterFactories() method.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394241322
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 
 Review comment:
   our code style unfortunately does not cover the placement of such braces; hence we reject any changes such as this to existing code.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386662377
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -273,3 +319,4 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return Optional.of((MetricReporter) reporterClass.newInstance());
 	}
 }
+
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155617305",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * d07ec693eb90e33071e710be0b774fc995f05867 Travis: [CANCELED](https://travis-ci.com/github/flink-ci/flink/builds/154179607) 
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * eccc41d96524bd8ff6bfa6bfa36f366689255893 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/155617305) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 3e2cdca58b30cb79c92bee7fc073dc0a494069ab Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/153707715) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r388340028
 
 

 ##########
 File path: flink-core/src/main/java/org/apache/flink/core/plugin/PluginLoader.java
 ##########
 @@ -69,7 +68,7 @@ public static PluginLoader create(PluginDescriptor pluginDescriptor, ClassLoader
 	 * @param <P> Type of the requested plugin service.
 	 * @return An iterator of all implementations of the given service interface that could be loaded from the plugin.
 	 */
-	public <P extends Plugin> Iterator<P> load(Class<P> service) {
+	public <P> Iterator<P> load(Class<P> service) {
 
 Review comment:
   That was actually my proposal, since there is no benefit from implementing Plugin and its `#configure` method does not work well with existing metric factories.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394321880
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -210,17 +213,31 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return namedOrderedReporters;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
-
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.info("Prepare reporter factories (from both SPIs and Plugins):");
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.info("Found reporter factory {} at {} ",
+						factoryClassName,
+						new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   I wouldn't care if it's just in a comment or internal exception. But if it's user facing, I'd strongly recommend to proof-read everything and fix it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393580453
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   Strictly speaking it then should be 'and/or', because 'or' could mean that there are either two jars both either in 'lib' or 'plugins'. I think and was more fitting, but I am also ok with and/or

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394180142
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -260,16 +262,26 @@ public void submitSQLJob(SQLJobSubmission job) throws IOException {
 	}
 
 	public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException {
-		final Optional<Path> reporterJarOptional;
-		try (Stream<Path> logFiles = Files.walk(opt)) {
-			reporterJarOptional = logFiles
+		copyOptJars(jarNamePrefix, lib);
+	}
+
+	public void copyOptJarsToPlugins(String jarNamePrefix) throws FileNotFoundException, IOException {
 
 Review comment:
   This will require a rebase; we added a more generic version for copying jars so we don't have to keep adding new ones. You will have to add a `JarLocation` for the plugins directory, and modify `FlinkDistribution#moveJar` to handle this location appropriately.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155617305",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155680617",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * 19b07382fedcab4e7c4866eb260a12cde4780538 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/155680617) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r388310058
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
 ##########
 @@ -160,13 +161,14 @@ public void startCluster() throws ClusterEntrypointException {
 		LOG.info("Starting {}.", getClass().getSimpleName());
 
 		try {
-
-			configureFileSystems(configuration);
+			//TODO: push down filesystem initialization into runCluster - initializeServices (?)
 
 Review comment:
   Just tagging TODO.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394246528
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
 
 Review comment:
   We usually don't allow commented code, and I agree with arvid that it would be easy to bring back.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 21c0efa3b9bae9e5bb23338c23483f41f0d1e738 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151445413) 
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386649079
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
+				}
 			} catch (Exception | ServiceConfigurationError e) {
 				LOG.warn("Error while loading reporter factory.", e);
 			}
 		}
-
 
 Review comment:
   Is this preferred style in Flink or just according to a principle of touching as few lines as possible? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7ce9ad87a780aab19b9c97680d7af5f648c13f3b Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/154170382) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r388344253
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 
 Review comment:
   In general, the best option is to leave old code as is to not blow up the PR. You could make a separate hotfix to address code style fixes though.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386675530
 
 

 ##########
 File path: flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java
 ##########
 @@ -35,7 +35,7 @@
  * {@link MetricReporter} that exports {@link Metric Metrics} via Prometheus.
  */
 @PublicEvolving
-public class PrometheusReporter extends AbstractPrometheusReporter {
+public class PrometheusReporter extends AbstractPrometheusReporter implements MetricReporter {
 
 Review comment:
   missed that AbstractPrometheusReporter already implements it. Fixed. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386646692
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   But this would technically be "multiple implementations of the same reporter"? The message does not tell explicitly that one of them is in lib and another in plugins - just that while searching those two directories, multiple implementations were found.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386469696
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
 
 Review comment:
   remove TODOs

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d693668c44effeffafc944835df873e3978f88b9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150196559) 
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r397676218
 
 

 ##########
 File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
 ##########
 @@ -120,24 +122,51 @@ public static void checkOS() {
 	public final DownloadCache downloadCache = DownloadCache.get();
 
 	@Test
-	public void testReporter() throws Exception {
-		dist.copyOptJarsToLib("flink-metrics-prometheus");
+	public void reporterWorksWhenFoundInLibsViaReflection() throws Exception {
+		dist.copyOptJarsToLib(PROMETHEUS_JAR_PREFIX);
 
 Review comment:
   I'd opt for creating a separate FlinkResource in every test.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394239114
 
 

 ##########
 File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
 ##########
 @@ -120,24 +122,51 @@ public static void checkOS() {
 	public final DownloadCache downloadCache = DownloadCache.get();
 
 	@Test
-	public void testReporter() throws Exception {
-		dist.copyOptJarsToLib("flink-metrics-prometheus");
+	public void reporterWorksWhenFoundInLibsViaReflection() throws Exception {
+		dist.copyOptJarsToLib(PROMETHEUS_JAR_PREFIX);
+		testReporter(false);
+	}
+
+	@Test
+	public void reporterWorksWhenFoundInPluginsViaReflection() throws Exception {
+		dist.copyOptJarsToPlugins(PROMETHEUS_JAR_PREFIX);
+		testReporter(false);
+	}
+
+	@Test
+	public void reporterWorksWhenFoundInPluginsViaFactories() throws Exception {
+		dist.copyOptJarsToPlugins(PROMETHEUS_JAR_PREFIX);
+		testReporter(true);
+	}
 
+	@Test
+	public void reporterWorksWhenFoundBothInPluginsAndLibsViaFactories() throws Exception {
+		dist.copyOptJarsToPlugins(PROMETHEUS_JAR_PREFIX);
+		dist.copyOptJarsToLib(PROMETHEUS_JAR_PREFIX);
+		testReporter(true);
+	}
+
+	private void testReporter(boolean useFactory) throws Exception {
 		final Configuration config = new Configuration();
-		config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+
+		if (useFactory) {
+			config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX, PrometheusReporterFactory.class.getName());
+		} else {
+			config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+		}
+
 		config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100");
 
 		dist.appendConfiguration(config);
 
 		final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus");
-		final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz");
 		final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME);
 		final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml");
 		final Path prometheusBinary = prometheusBinDir.resolve("prometheus");
 		Files.createDirectory(tmpPrometheusDir);
 
-		downloadCache.getOrDownload(
-			"https://github.com/prometheus/prometheus/releases/download/v" + PROMETHEUS_VERSION + '/' + prometheusArchive.getFileName(),
+		final Path prometheusArchive = downloadCache.getOrDownload(
 
 Review comment:
   this belongs into a separate commit since it is fixing a bug in the test that can occur independently from this PR.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393563258
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
 ##########
 @@ -160,13 +161,14 @@ public void startCluster() throws ClusterEntrypointException {
 		LOG.info("Starting {}.", getClass().getSimpleName());
 
 		try {
-
-			configureFileSystems(configuration);
+			//TODO: push down filesystem initialization into runCluster - initializeServices (?)
 
 Review comment:
   @AHeise I wanted to ask if what is written in TODO is a good idea in your opinion. I am not sure about the implications of initializing FileSystems within `runSecured`. I do not quite like that initialization of services (initializeServices) and  file systems happen in different places. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393950508
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 		String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, "");
-		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
-			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
-			.collect(Collectors.toSet());
 
-		// use a TreeSet to make the reporter order deterministic, which is useful for testing
-		Set<String> namedReporters = new TreeSet<>(String::compareTo);
-		// scan entire configuration for "metric.reporter" keys and parse individual reporter configurations
-		for (String key : configuration.keySet()) {
-			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
-				Matcher matcher = reporterClassPattern.matcher(key);
-				if (matcher.matches()) {
-					String reporterName = matcher.group(1);
-					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
-						if (namedReporters.contains(reporterName)) {
-							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
-						} else {
-							namedReporters.add(reporterName);
-						}
-					} else {
-						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
-					}
-				}
-			}
-		}
+		Set<String> namedReporters = findEnabledReportersInConfiguration(configuration,
+			includedReportersString);
 
 		if (namedReporters.isEmpty()) {
 			return Collections.emptyList();
 		}
 
-		List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+		final Map<String, MetricReporterFactory> reporterFactories = loadAvailableReporterFactories(pluginManager);
+		LOG.debug("Loaded Reporter Factories: {}", reporterFactories);
+		final List<Tuple2<String, Configuration>> reporterConfigurations = loadReporterConfigurations(configuration, namedReporters);
+		LOG.debug("Loaded Reporter Configurations: {}", reporterConfigurations);
 
-		for (String namedReporter: namedReporters) {
-			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
-				configuration,
-				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+		List<ReporterSetup> reporterSetups = setupReporters(reporterFactories, reporterConfigurations);
+		LOG.debug("All initialized Reporters:");
+		reporterSetups.forEach(i -> LOG.debug("{} - {}", i.getName(), i.getConfiguration()));
 
 Review comment:
   Addressed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386472702
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
 
 Review comment:
   Should likely be removed or replaced with a meaningful INFO message that a factory was found.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386476291
 
 

 ##########
 File path: flink-core/src/main/java/org/apache/flink/core/plugin/PluginLoader.java
 ##########
 @@ -69,7 +68,7 @@ public static PluginLoader create(PluginDescriptor pluginDescriptor, ClassLoader
 	 * @param <P> Type of the requested plugin service.
 	 * @return An iterator of all implementations of the given service interface that could be loaded from the plugin.
 	 */
-	public <P extends Plugin> Iterator<P> load(Class<P> service) {
+	public <P> Iterator<P> load(Class<P> service) {
 
 Review comment:
   ping @pnowojski / @AHeise 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 3aded2be8a0019e7b025d811d204e92d36dd24e7 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/153641242) 
   * 57db59bd20c14d5a0872f6d35b2b3220624ec96b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395365114
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -210,17 +213,31 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return namedOrderedReporters;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
-
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.info("Prepare reporter factories (from both SPIs and Plugins):");
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.info("Found reporter factory {} at {} ",
+						factoryClassName,
+						new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   I thing the problem is that we have too many variants to describe with a single coordinating conjunction like "or" (plus "or" can unfortunately be both inclusive and exclusive). Cases:
   (xx) _ ()
   () _ (xx)
   (x) _ (x)
   (xx) _ (xx)
   
   Alternatives like "a or b or both" also do not work, because in this particular case they can be easily be interpreted as if the "(x) _ (x)" case is not an issue (because of "Multiple" in the beginning). Seems to me like the best case to indicate this ambiguity is is to use "and/or" so that people will be aware for what kinds of problem potentially to look for.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 57db59bd20c14d5a0872f6d35b2b3220624ec96b Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/153672077) 
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386474188
 
 

 ##########
 File path: flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java
 ##########
 @@ -35,7 +35,7 @@
  * {@link MetricReporter} that exports {@link Metric Metrics} via Prometheus.
  */
 @PublicEvolving
-public class PrometheusReporter extends AbstractPrometheusReporter {
+public class PrometheusReporter extends AbstractPrometheusReporter implements MetricReporter {
 
 Review comment:
   why is this necessary?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386477670
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 
 Review comment:
   we should actually remove this line (or exclude the metricConfig) as we may be leaking sensitive information.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386649079
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
+				}
 			} catch (Exception | ServiceConfigurationError e) {
 				LOG.warn("Error while loading reporter factory.", e);
 			}
 		}
-
 
 Review comment:
   Is this accepted Flink style or just according to a principle of touching as few lines as possible? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7ce9ad87a780aab19b9c97680d7af5f648c13f3b Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/154170382) 
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * d07ec693eb90e33071e710be0b774fc995f05867 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/154179607) 
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r397030268
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   You've mentioned "to get rid of the next few lines", but this Tree data structure is further used as a container that is filled with some conditional logic and returned, it is not just about having the input entries sorted. Do you propose to rewrite it?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393903505
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 
 Review comment:
   > do we have a Flink-wide way to handle such cases
   
   The `GlobalConfiguration` contains a set of keys that are considered sensitive, which we use for the WebUI and various INFO logging.
   
   > We have been "leaking" this data in the current versions with info (!) level without much concern
   
   Few reporters actually use credentials (afaik only datadog does), so the sample size is fairly low.
   
   > If someone has uncontrolled access to the log files on your machine in production, content of this file is probably not the biggest of your problems.
   
   Doesn't need access to the machine; access to the UI is sufficient, which was grave enough that we introduced the whole secret-key concept in the first place.
   
   > Running with debug level is not a "normal" scenario - this is intended for hands on investigation of issues. Log level for potentially compromisable external systems could be explicitly set to trace in such cases.
   
   There's precedence with FLINK-10363 that credentials should not be logged even on debug.
   FLINK-16478 also which proposes a REST API for modifying the log level potentially voiding any argument for it being opt-in insecurity.
   
   I would approach this cautiously and never log anything sensitive.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r399084928
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceSetup.java
 ##########
 @@ -69,13 +69,13 @@ public FlinkResourceSetupBuilder addConfiguration(Configuration config) {
 			return this;
 		}
 
-		public FlinkResourceSetupBuilder moveJar(String jarNamePrefix, JarLocation source, JarLocation target) {
-			this.jarMoveOperations.add(new JarMove(jarNamePrefix, source, target));
+		public FlinkResourceSetupBuilder addJarOperation(JarOperation jarOperation) {
+			this.jarOperations.add(jarOperation);
 			return this;
 		}
 
 		public FlinkResourceSetup build() {
-			return new FlinkResourceSetup(config, Collections.unmodifiableCollection(jarMoveOperations));
+			return new FlinkResourceSetup(config,  Collections.unmodifiableCollection(jarOperations));
 
 Review comment:
   ```suggestion
   			return new FlinkResourceSetup(config, Collections.unmodifiableCollection(jarOperations));
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393563258
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
 ##########
 @@ -160,13 +161,14 @@ public void startCluster() throws ClusterEntrypointException {
 		LOG.info("Starting {}.", getClass().getSimpleName());
 
 		try {
-
-			configureFileSystems(configuration);
+			//TODO: push down filesystem initialization into runCluster - initializeServices (?)
 
 Review comment:
   @AHeise I wanted to ask if what is written in TODO is a good idea in your opinion. I am not sure about the implications of initializing FileSystems within `runSecured`. I do not quite like that initialization if services (initializeServices) and  file systems happen in different places. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r397036620
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   Ah you are right. Ignore my comment.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394236320
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
 ##########
 @@ -160,13 +161,14 @@ public void startCluster() throws ClusterEntrypointException {
 		LOG.info("Starting {}.", getClass().getSimpleName());
 
 		try {
-
-			configureFileSystems(configuration);
+			//TODO: push down filesystem initialization into runCluster - initializeServices (?)
 
 Review comment:
   I've got no clue. Maybe @tillrohrmann remembers why this was added outside `runSecured` in https://github.com/apache/flink/commit/bbac4a6c922199db08a5244d0fa1262a5f16d479#diff-5334e24ac6a0d7e69599ceca71fd2e99.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386662966
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386649079
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
+				}
 			} catch (Exception | ServiceConfigurationError e) {
 				LOG.warn("Error while loading reporter factory.", e);
 			}
 		}
-
 
 Review comment:
   Is this a preferred style in Flink or just according to a principle of touching as few lines as possible? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r394183150
 
 

 ##########
 File path: flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporterFactory.java
 ##########
 @@ -0,0 +1,34 @@
+/*
+ * 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.metrics.prometheus;
+
+import org.apache.flink.core.plugin.Plugin;
+import org.apache.flink.metrics.reporter.MetricReporterFactory;
+
+import java.util.Properties;
+
+/**
+ * {@link MetricReporterFactory} for {@link PrometheusReporter}.
+ */
+public class PrometheusReporterFactory implements MetricReporterFactory, Plugin {
+
+	@Override
+	public PrometheusReporter createMetricReporter(Properties properties) {
+		return new PrometheusReporter();
 
 Review comment:
   Ideally we move the logic from `PrometheusReporter#open` into this method and change the constructor accordingly.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386467901
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerTest.java
 ##########
 @@ -92,7 +92,7 @@ private static Configuration createConfiguration() {
 	}
 
 	private static TaskManagerRunner createTaskManagerRunner(final Configuration configuration) throws Exception {
-		TaskManagerRunner taskManagerRunner = new TaskManagerRunner(configuration, ResourceID.generate());
+		TaskManagerRunner taskManagerRunner = new TaskManagerRunner(configuration, ResourceID.generate(), null);
 
 Review comment:
   why is this passing `null` and not following the same approach as `JobManagerHAProcessFailureRecoveryITCase`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7ce9ad87a780aab19b9c97680d7af5f648c13f3b Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/154170382) 
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393578320
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -77,6 +77,7 @@
 	private Path conf;
 
 Review comment:
   Addressed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r396809894
 
 

 ##########
 File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
 ##########
 @@ -120,24 +122,51 @@ public static void checkOS() {
 	public final DownloadCache downloadCache = DownloadCache.get();
 
 	@Test
-	public void testReporter() throws Exception {
-		dist.copyOptJarsToLib("flink-metrics-prometheus");
+	public void reporterWorksWhenFoundInLibsViaReflection() throws Exception {
+		dist.copyOptJarsToLib(PROMETHEUS_JAR_PREFIX);
 
 Review comment:
   @zentol It seems that this updated approach that got merged into master does not support the kinds of tests that we would need to do for the supported scenarios (see reporterWorksWhenFoundInLibsViaReflection, reporterWorksWhenFoundInPluginsViaReflection, reporterWorksWhenFoundBothInPluginsAndLibsViaFactories in this PR). How should we proceed? I can either bend the FlinkResource implementation back to the state where it supports modifications of the underlying resources after creation and keep the initialization in @Rule (a hack), or reinitizliae FlinkResource in every test (probably a bad idea from performance and overhead perspectives). What do you think?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393950683
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 
 Review comment:
   Removed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395375550
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -260,16 +262,26 @@ public void submitSQLJob(SQLJobSubmission job) throws IOException {
 	}
 
 	public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException {
-		final Optional<Path> reporterJarOptional;
-		try (Stream<Path> logFiles = Files.walk(opt)) {
-			reporterJarOptional = logFiles
+		copyOptJars(jarNamePrefix, lib);
+	}
+
+	public void copyOptJarsToPlugins(String jarNamePrefix) throws FileNotFoundException, IOException {
 
 Review comment:
   > and modify FlinkDistribution#moveJar to handle this location appropriately.
   
   could you clarify? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395348055
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   @AHeise I have applied this refactoring but then understood that I probably did not get what you actually propose. Could you please clarify? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r396809894
 
 

 ##########
 File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
 ##########
 @@ -120,24 +122,51 @@ public static void checkOS() {
 	public final DownloadCache downloadCache = DownloadCache.get();
 
 	@Test
-	public void testReporter() throws Exception {
-		dist.copyOptJarsToLib("flink-metrics-prometheus");
+	public void reporterWorksWhenFoundInLibsViaReflection() throws Exception {
+		dist.copyOptJarsToLib(PROMETHEUS_JAR_PREFIX);
 
 Review comment:
   @zentol It seems that this updated approach that got merged into master does not support the kinds of tests that we would need to do for the supported scenarios (see reporterWorksWhenFoundInLibsViaReflection, reporterWorksWhenFoundInPluginsViaReflection, reporterWorksWhenFoundBothInPluginsAndLibsViaFactories in this PR). How should we proceed? I can either bend the FlinkResource implementation back to the state where it supports modifications of the underlying resources after creation and keep the initialization in `@Rule` (a hack), or reinitialize FlinkResource in every test. What do you think?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393840337
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 		String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, "");
-		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
 
 Review comment:
   Done.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393904732
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
 
 Review comment:
   This would flood the logs with warnings for every single configured parameter, as they all have the `metrics.reporter.` prefix but don't end with `class`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386682800
 
 

 ##########
 File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
 ##########
 @@ -185,6 +214,7 @@ public void testReporter() throws Exception {
 
 			checkMetricAvailability(client, "flink_jobmanager_numRegisteredTaskManagers");
 			checkMetricAvailability(client, "flink_taskmanager_Status_Network_TotalMemorySegments");
+
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386471891
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   Is it guaranteed that they are in opt _and_ lib? What if 2 plugins specified the same factory class?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r399084072
 
 

 ##########
 File path: flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory
 ##########
 @@ -0,0 +1,16 @@
+# 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.
+
+org.apache.flink.metrics.prometheus.PrometheusReporterFactory
 
 Review comment:
   ```suggestion
   org.apache.flink.metrics.prometheus.PrometheusReporterFactory
   org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterFactory
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7ce9ad87a780aab19b9c97680d7af5f648c13f3b Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/154170382) 
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * d07ec693eb90e33071e710be0b774fc995f05867 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393832500
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 
 Review comment:
   This code was touched anyhow, because of refactoring, so I think it should be OK to do such things, unless you have a strong opinion.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 57db59bd20c14d5a0872f6d35b2b3220624ec96b Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/153672077) 
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 3e2cdca58b30cb79c92bee7fc073dc0a494069ab UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d693668c44effeffafc944835df873e3978f88b9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 3e2cdca58b30cb79c92bee7fc073dc0a494069ab Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/153707715) 
   * a244cee274bab74683c25f90bfd515698bc04c95 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155617305",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * eccc41d96524bd8ff6bfa6bfa36f366689255893 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/155617305) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691) 
   * 19b07382fedcab4e7c4866eb260a12cde4780538 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395617873
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -260,16 +262,26 @@ public void submitSQLJob(SQLJobSubmission job) throws IOException {
 	}
 
 	public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException {
-		final Optional<Path> reporterJarOptional;
-		try (Stream<Path> logFiles = Files.walk(opt)) {
-			reporterJarOptional = logFiles
+		copyOptJars(jarNamePrefix, lib);
+	}
+
+	public void copyOptJarsToPlugins(String jarNamePrefix) throws FileNotFoundException, IOException {
 
 Review comment:
   hmmm that's a bit frustrating; now we have to open up the API to allow things that people shouldn't be doing; though admittedly we should be able to test everything a user _might_ do.
   
   A dumb intermediate workaround would be to have 2 variants; one for copying/moving each.
   
   You will also have to modify `#moveJar` to pass in the file(name at least) to `#mapJarLocationPath` so that you can introduce the directory for the individual plugin.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386479500
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 		String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, "");
-		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
-			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
-			.collect(Collectors.toSet());
 
-		// use a TreeSet to make the reporter order deterministic, which is useful for testing
-		Set<String> namedReporters = new TreeSet<>(String::compareTo);
-		// scan entire configuration for "metric.reporter" keys and parse individual reporter configurations
-		for (String key : configuration.keySet()) {
-			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
-				Matcher matcher = reporterClassPattern.matcher(key);
-				if (matcher.matches()) {
-					String reporterName = matcher.group(1);
-					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
-						if (namedReporters.contains(reporterName)) {
-							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
-						} else {
-							namedReporters.add(reporterName);
-						}
-					} else {
-						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
-					}
-				}
-			}
-		}
+		Set<String> namedReporters = findEnabledReportersInConfiguration(configuration,
+			includedReportersString);
 
 		if (namedReporters.isEmpty()) {
 			return Collections.emptyList();
 		}
 
-		List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+		final Map<String, MetricReporterFactory> reporterFactories = loadAvailableReporterFactories(pluginManager);
+		LOG.debug("Loaded Reporter Factories: {}", reporterFactories);
+		final List<Tuple2<String, Configuration>> reporterConfigurations = loadReporterConfigurations(configuration, namedReporters);
+		LOG.debug("Loaded Reporter Configurations: {}", reporterConfigurations);
 
 Review comment:
   same as above about leaking sensitive information

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r397030268
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   You've mentioned "to get rid of the next few lines", but this Tree data structure is further used as a container that is filled using some conditional logic and returned, it is not just about having the input entries sorted. Do you propose to rewrite it?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d693668c44effeffafc944835df873e3978f88b9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150196559) 
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 1613baf0244adc63bf029415e97227a6770591c9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393843746
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java
 ##########
 @@ -202,9 +204,11 @@ private SecurityContext installSecurityContext(Configuration configuration) thro
 		return SecurityUtils.getInstalledContext();
 	}
 
-	private void runCluster(Configuration configuration) throws Exception {
+	private void runCluster(Configuration configuration, PluginManager pluginManager) throws Exception {
 		synchronized (lock) {
-			initializeServices(configuration);
+
+			//TODO: Ask why FileSystem is not initialized here too.
 
 Review comment:
   See above.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * a244cee274bab74683c25f90bfd515698bc04c95 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/153812547) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395376334
 
 

 ##########
 File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
 ##########
 @@ -260,16 +262,26 @@ public void submitSQLJob(SQLJobSubmission job) throws IOException {
 	}
 
 	public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException {
-		final Optional<Path> reporterJarOptional;
-		try (Stream<Path> logFiles = Files.walk(opt)) {
-			reporterJarOptional = logFiles
+		copyOptJars(jarNamePrefix, lib);
+	}
+
+	public void copyOptJarsToPlugins(String jarNamePrefix) throws FileNotFoundException, IOException {
 
 Review comment:
   I assume you mean to modify `FlinkDistribution#mapJarLocationToPath` ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386479537
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 		String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, "");
-		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
-			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
-			.collect(Collectors.toSet());
 
-		// use a TreeSet to make the reporter order deterministic, which is useful for testing
-		Set<String> namedReporters = new TreeSet<>(String::compareTo);
-		// scan entire configuration for "metric.reporter" keys and parse individual reporter configurations
-		for (String key : configuration.keySet()) {
-			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
-				Matcher matcher = reporterClassPattern.matcher(key);
-				if (matcher.matches()) {
-					String reporterName = matcher.group(1);
-					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
-						if (namedReporters.contains(reporterName)) {
-							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
-						} else {
-							namedReporters.add(reporterName);
-						}
-					} else {
-						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
-					}
-				}
-			}
-		}
+		Set<String> namedReporters = findEnabledReportersInConfiguration(configuration,
+			includedReportersString);
 
 		if (namedReporters.isEmpty()) {
 			return Collections.emptyList();
 		}
 
-		List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+		final Map<String, MetricReporterFactory> reporterFactories = loadAvailableReporterFactories(pluginManager);
+		LOG.debug("Loaded Reporter Factories: {}", reporterFactories);
+		final List<Tuple2<String, Configuration>> reporterConfigurations = loadReporterConfigurations(configuration, namedReporters);
+		LOG.debug("Loaded Reporter Configurations: {}", reporterConfigurations);
 
-		for (String namedReporter: namedReporters) {
-			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
-				configuration,
-				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+		List<ReporterSetup> reporterSetups = setupReporters(reporterFactories, reporterConfigurations);
+		LOG.debug("All initialized Reporters:");
+		reporterSetups.forEach(i -> LOG.debug("{} - {}", i.getName(), i.getConfiguration()));
 
 Review comment:
   same as above about leaking sensitive information

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395323032
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 
 Review comment:
   Done.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r396809894
 
 

 ##########
 File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
 ##########
 @@ -120,24 +122,51 @@ public static void checkOS() {
 	public final DownloadCache downloadCache = DownloadCache.get();
 
 	@Test
-	public void testReporter() throws Exception {
-		dist.copyOptJarsToLib("flink-metrics-prometheus");
+	public void reporterWorksWhenFoundInLibsViaReflection() throws Exception {
+		dist.copyOptJarsToLib(PROMETHEUS_JAR_PREFIX);
 
 Review comment:
   @zentol It seems that this updated approach that got merged into master does not support the kinds of tests that we would need to do for the supported scenarios (see reporterWorksWhenFoundInLibsViaReflection, reporterWorksWhenFoundInPluginsViaReflection, reporterWorksWhenFoundBothInPluginsAndLibsViaFactories in this PR). How should we proceed? I can either bend the FlinkResource implementation back to the state where it supports modifications of the underlying resources after creation and keep the initialization in `@Rule` (a hack), or reinitizliae FlinkResource in every test (probably a bad idea from performance and overhead perspectives). What do you think?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386478652
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 		String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, "");
-		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
 
 Review comment:
   reviews are a lot easier if we either a) refrain from non-critical refactorings b) move such refactorings into a separate commit.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395360427
 
 

 ##########
 File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
 ##########
 @@ -120,24 +122,51 @@ public static void checkOS() {
 	public final DownloadCache downloadCache = DownloadCache.get();
 
 	@Test
-	public void testReporter() throws Exception {
-		dist.copyOptJarsToLib("flink-metrics-prometheus");
+	public void reporterWorksWhenFoundInLibsViaReflection() throws Exception {
+		dist.copyOptJarsToLib(PROMETHEUS_JAR_PREFIX);
+		testReporter(false);
+	}
+
+	@Test
+	public void reporterWorksWhenFoundInPluginsViaReflection() throws Exception {
+		dist.copyOptJarsToPlugins(PROMETHEUS_JAR_PREFIX);
+		testReporter(false);
+	}
+
+	@Test
+	public void reporterWorksWhenFoundInPluginsViaFactories() throws Exception {
+		dist.copyOptJarsToPlugins(PROMETHEUS_JAR_PREFIX);
+		testReporter(true);
+	}
 
+	@Test
+	public void reporterWorksWhenFoundBothInPluginsAndLibsViaFactories() throws Exception {
+		dist.copyOptJarsToPlugins(PROMETHEUS_JAR_PREFIX);
+		dist.copyOptJarsToLib(PROMETHEUS_JAR_PREFIX);
+		testReporter(true);
+	}
+
+	private void testReporter(boolean useFactory) throws Exception {
 		final Configuration config = new Configuration();
-		config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+
+		if (useFactory) {
+			config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX, PrometheusReporterFactory.class.getName());
+		} else {
+			config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+		}
+
 		config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100");
 
 		dist.appendConfiguration(config);
 
 		final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus");
-		final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz");
 		final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME);
 		final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml");
 		final Path prometheusBinary = prometheusBinDir.resolve("prometheus");
 		Files.createDirectory(tmpPrometheusDir);
 
-		downloadCache.getOrDownload(
-			"https://github.com/prometheus/prometheus/releases/download/v" + PROMETHEUS_VERSION + '/' + prometheusArchive.getFileName(),
+		final Path prometheusArchive = downloadCache.getOrDownload(
 
 Review comment:
   Split as requested.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386469740
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
+				}
 			} catch (Exception | ServiceConfigurationError e) {
 				LOG.warn("Error while loading reporter factory.", e);
 			}
 		}
-
 
 Review comment:
   revert

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393580453
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					String jarPath2 = new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+//						.toURI()).getCanonicalPath();
+//					LOG.warn("Multiple implementations of the same reporter were found: \n {} and \n{}", jarPath1, jarPath2);
+					LOG.warn("Multiple implementations of the same reporter were found in 'lib' and 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName);
 
 Review comment:
   Strictly speaking it then should be 'and/or', because 'or' could mean that there are two jars, both of which are either in 'lib' or in 'plugins'. I think and was more fitting, but I am also ok with and/or

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 1613baf0244adc63bf029415e97227a6770591c9 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151434781) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-593650822
 
 
   Thanks, @zentol, I have addressed your comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386620764
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerTest.java
 ##########
 @@ -92,7 +92,7 @@ private static Configuration createConfiguration() {
 	}
 
 	private static TaskManagerRunner createTaskManagerRunner(final Configuration configuration) throws Exception {
-		TaskManagerRunner taskManagerRunner = new TaskManagerRunner(configuration, ResourceID.generate());
+		TaskManagerRunner taskManagerRunner = new TaskManagerRunner(configuration, ResourceID.generate(), null);
 
 Review comment:
   adjusted

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155617305",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * eccc41d96524bd8ff6bfa6bfa36f366689255893 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/155617305) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155617305",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6691",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/155680617",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699",
       "triggerID" : "19b07382fedcab4e7c4866eb260a12cde4780538",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a088a5f122ccce53e5baa97603eb8b0a9599247",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * 19b07382fedcab4e7c4866eb260a12cde4780538 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/155680617) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6699) 
   * 6a088a5f122ccce53e5baa97603eb8b0a9599247 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r395368073
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -178,36 +164,96 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
+
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
 
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
+
+		// use a TreeSet to make the reporter order deterministic, which is useful for testing
+		Set<String> namedOrderedReporters = new TreeSet<>(String::compareTo);
+
+		// scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters
+		for (String key : configuration.keySet()) {
+			if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) {
+				Matcher matcher = reporterClassPattern.matcher(key);
+				if (matcher.matches()) {
+					String reporterName = matcher.group(1);
+					if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) {
+						if (namedOrderedReporters.contains(reporterName)) {
+							LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName);
+						} else {
+							namedOrderedReporters.add(reporterName);
+						}
+					} else {
+						LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString);
+					}
+				}
+			}
+		}
+		return namedOrderedReporters;
+	}
+
+	private static Map<String, MetricReporterFactory> loadAvailableReporterFactories(PluginManager pluginManager) {
 		final Map<String, MetricReporterFactory> reporterFactories = new HashMap<>(2);
-		final Iterator<MetricReporterFactory> factoryIterator = serviceLoader.iterator();
+		final Iterator<MetricReporterFactory> factoryIterator = getAllReporterFactories(pluginManager);
+		LOG.debug("All available factories (from both SPIs and Plugins):");
+		getAllReporterFactories(pluginManager).forEachRemaining(i -> LOG.debug(i.toString()));
 		// do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors
 		// such an error might be caused if the META-INF/services contains an entry to a non-existing factory class
 		while (factoryIterator.hasNext()) {
 			try {
 				MetricReporterFactory factory = factoryIterator.next();
-				reporterFactories.put(factory.getClass().getName(), factory);
+				String factoryClassName = factory.getClass().getName();
+				MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName);
+				if (existingFactory == null){
+					reporterFactories.put(factoryClassName, factory);
+					LOG.warn(new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation()
+						.toURI()).getCanonicalPath());
+				} else {
+					//TODO: use path information below, when Plugin Classloader stops always prioritizing factories from /lib
+//					String jarPath1 = new File(existingFactory.getClass().getProtectionDomain().getCodeSource().getLocation()
 
 Review comment:
   Removed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386474591
 
 

 ##########
 File path: flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java
 ##########
 @@ -35,7 +35,7 @@
  * {@link MetricReporter} that exports {@link Metric Metrics} via Prometheus.
  */
 @PublicEvolving
 
 Review comment:
   Add `@InstantiateViaFactory` annotation so that all instantiations go through the factory.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386644501
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
 ##########
 @@ -119,7 +120,7 @@
 
 	private boolean shutdown;
 
-	public TaskManagerRunner(Configuration configuration, ResourceID resourceId) throws Exception {
+	public TaskManagerRunner(Configuration configuration, ResourceID resourceId, PluginManager pluginManager) throws Exception {
 
 Review comment:
   I am  not sure what should be the expected contract here. ReporterSetup will currently only work with the null pluginManager if no reporters are configured (namedReporters.isEmpty()). We could maybe add a checkNonNull in  the ReporterSetup after that first return due to empty namedReporters.
   
   (TaskManagerRunnerTest is now fixed to initialize the PluginManager properly) 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393950810
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -120,55 +124,37 @@ public static ReporterSetup forReporter(String reporterName, MetricConfig metric
 	}
 
 	private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) {
-		LOG.info("Configuring {} with {}.", reporterName, metricConfig);
+		LOG.debug("Configuring {} with {}.", reporterName, metricConfig);
 		reporter.open(metricConfig);
 
 		return new ReporterSetup(reporterName, metricConfig, reporter);
 	}
 
-	public static List<ReporterSetup> fromConfiguration(final Configuration configuration) {
+	public static List<ReporterSetup> fromConfiguration(final Configuration configuration, final PluginManager pluginManager) {
+		LOG.debug("Initializing Reporters from Configuration: {}", configuration);
 
 Review comment:
   Removed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 21c0efa3b9bae9e5bb23338c23483f41f0d1e738 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/151445413) 
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 3aded2be8a0019e7b025d811d204e92d36dd24e7 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/153641242) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r396968257
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -179,28 +164,82 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 				metricReporterOptional.ifPresent(reporter -> {
 					MetricConfig metricConfig = new MetricConfig();
 					reporterConfig.addAllToProperties(metricConfig);
-
-					reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter));
+					reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter));
 				});
-			}
-			catch (Throwable t) {
+			} catch (Throwable t) {
 				LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t);
 			}
 		}
-		return reporterArguments;
+		return reporterSetups;
 	}
 
-	private static Map<String, MetricReporterFactory> loadReporterFactories() {
-		final ServiceLoader<MetricReporterFactory> serviceLoader = ServiceLoader.load(MetricReporterFactory.class);
+	private static List<Tuple2<String, Configuration>> loadReporterConfigurations(Configuration configuration, Set<String> namedReporters) {
+		final List<Tuple2<String, Configuration>> reporterConfigurations = new ArrayList<>(namedReporters.size());
 
+		for (String namedReporter: namedReporters) {
+			DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration(
+				configuration,
+				ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + '.');
+
+			reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration));
+		}
+		return reporterConfigurations;
+	}
+
+	private static Set<String> findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) {
+		Set<String> includedReporters = reporterListPattern.splitAsStream(includedReportersString)
+			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
+			.collect(Collectors.toSet());
 
 Review comment:
   ```
   Set<String> namedOrderedReporters = reporterListPattern.splitAsStream(includedReportersString)
   			.filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+
   			.collect(Collectors.toCollection(TreeSet::new));
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
afedulov commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r393256771
 
 

 ##########
 File path: flink-core/src/main/java/org/apache/flink/core/plugin/PluginLoader.java
 ##########
 @@ -69,7 +68,7 @@ public static PluginLoader create(PluginDescriptor pluginDescriptor, ClassLoader
 	 * @param <P> Type of the requested plugin service.
 	 * @return An iterator of all implementations of the given service interface that could be loaded from the plugin.
 	 */
-	public <P extends Plugin> Iterator<P> load(Class<P> service) {
+	public <P> Iterator<P> load(Class<P> service) {
 
 Review comment:
   Marking as resolved.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#discussion_r386469003
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 ##########
 @@ -273,3 +319,4 @@ private static ReporterSetup createReporterSetup(String reporterName, MetricConf
 		return Optional.of((MetricReporter) reporterClass.newInstance());
 	}
 }
+
 
 Review comment:
   revert

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11195: [FLINK-16222][runtime] Use plugins mechanism for initializing MetricReporters
URL: https://github.com/apache/flink/pull/11195#issuecomment-590065588
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d693668c44effeffafc944835df873e3978f88b9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150196559",
       "triggerID" : "d693668c44effeffafc944835df873e3978f88b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "738a6fd162d5ba99432438d224f3b4890961af56",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1613baf0244adc63bf029415e97227a6770591c9",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151434781",
       "triggerID" : "1613baf0244adc63bf029415e97227a6770591c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/151445413",
       "triggerID" : "21c0efa3b9bae9e5bb23338c23483f41f0d1e738",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77f24b292a04fde34f4c645151484cf86a13832c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dfb306ba6ff07f700b1afc21847641321705d4d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153641242",
       "triggerID" : "3aded2be8a0019e7b025d811d204e92d36dd24e7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153672077",
       "triggerID" : "57db59bd20c14d5a0872f6d35b2b3220624ec96b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b0f219689e81fe11c3511c13c7f4943a997a4ef",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153707715",
       "triggerID" : "3e2cdca58b30cb79c92bee7fc073dc0a494069ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/153812547",
       "triggerID" : "a244cee274bab74683c25f90bfd515698bc04c95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154170382",
       "triggerID" : "7ce9ad87a780aab19b9c97680d7af5f648c13f3b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f6ad172474e37bbf346a7762868c9114510c4c9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2590a69417bdd2b6bf2f4765b276a051040a97cf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/154179607",
       "triggerID" : "d07ec693eb90e33071e710be0b774fc995f05867",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a1b15671c97e8809c0301fb4c225fdc1a7f78545",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eccc41d96524bd8ff6bfa6bfa36f366689255893",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 738a6fd162d5ba99432438d224f3b4890961af56 UNKNOWN
   * 77f24b292a04fde34f4c645151484cf86a13832c UNKNOWN
   * 5dfb306ba6ff07f700b1afc21847641321705d4d UNKNOWN
   * 5b0f219689e81fe11c3511c13c7f4943a997a4ef UNKNOWN
   * 7f6ad172474e37bbf346a7762868c9114510c4c9 UNKNOWN
   * 2590a69417bdd2b6bf2f4765b276a051040a97cf UNKNOWN
   * d07ec693eb90e33071e710be0b774fc995f05867 Travis: [CANCELED](https://travis-ci.com/github/flink-ci/flink/builds/154179607) 
   * a1b15671c97e8809c0301fb4c225fdc1a7f78545 UNKNOWN
   * eccc41d96524bd8ff6bfa6bfa36f366689255893 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services