You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by xi...@apache.org on 2024/02/08 17:14:18 UTC

(pinot) branch master updated: Adding support for configuration through environment variables (#12307)

This is an automated email from the ASF dual-hosted git repository.

xiangfu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 7c9bf8cf49 Adding support for configuration through environment variables (#12307)
7c9bf8cf49 is described below

commit 7c9bf8cf49b0ed78f1de26edffa602bd3f74f548
Author: Tommaso Peresson <to...@team.bumble.com>
AuthorDate: Thu Feb 8 17:14:13 2024 +0000

    Adding support for configuration through environment variables (#12307)
    
    * init
    
    * Added support for env vars
    
    * Revert removal of SystemEnvironment and Environment classes
    
    * Fix tests
    
    * Fix formatting
    
    * Revert unnecessary changes
    
    * Revert unnecessary changes
    
    * Formatting
    
    * Comments
    
    * Tests fix
    
    * Added back legacy prefix
    
    * Fix test
    
    * Implementing dynamic templating for env config
    
    * Fix typo
---
 .../apache/pinot/spi/env/PinotConfiguration.java   | 83 ++++++++++++++--------
 .../pinot/spi/env/PinotConfigurationTest.java      | 44 ++++++++----
 .../resources/pinot-configuration-4.properties     | 23 ++++++
 3 files changed, 109 insertions(+), 41 deletions(-)

diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/env/PinotConfiguration.java b/pinot-spi/src/main/java/org/apache/pinot/spi/env/PinotConfiguration.java
index 6b7e9089eb..0516ee6b41 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/env/PinotConfiguration.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/env/PinotConfiguration.java
@@ -48,7 +48,8 @@ import org.apache.pinot.spi.utils.Obfuscator;
  * <li>Apache Commons {@link Configuration} (see {@link #PinotConfiguration(Configuration)})</li>
  * <li>Generic key-value properties provided with a {@link Map} (see
  * {@link PinotConfiguration#PinotConfiguration(Map)}</li>
- * <li>Environment variables (see {@link PinotConfiguration#PinotConfiguration(Map, Map)}</li>
+ * <li>Environment variables through env.dynamic.config (see {@link PinotConfiguration#PinotConfiguration(Map, Map)}
+ * </li>
  * <li>{@link PinotFSSpec} (see {@link PinotConfiguration#PinotConfiguration(PinotFSSpec)}</li>
  * </ul>
  * </p>
@@ -56,10 +57,16 @@ import org.apache.pinot.spi.utils.Obfuscator;
  * These different sources will all merged in an underlying {@link CompositeConfiguration} for which all
  * configuration keys will have
  * been sanitized.
- * Through this mechanism, properties can be configured and retrieved with kebab case, camel case, snake case and
- * environment variable
+ * Through this mechanism, properties can be configured and retrieved with kebab case, camel case and snake case
  * conventions.
  * </p>
+ * <strong>Dynamic configuration</strong>
+ * <p>
+ * In order to enable loading configurations through environment variables you can specify
+ * {@value ENV_DYNAMIC_CONFIG_KEY} as a list of property keys to dynamically template.
+ * {@link PinotConfiguration#applyDynamicEnvConfig(List, Map)}}. This enables loading secrets safely
+ * into the configuration.
+ * <p/>
  * <table>
  * <tr>
  * <th>Property</th>
@@ -77,15 +84,12 @@ import org.apache.pinot.spi.utils.Obfuscator;
  * <td>controller.sub_module.alerts.email_address</td>
  * <td>Snake case notation, which is an alternative format for use in .properties and .yml files.</td>
  * </tr>
- * <tr>
- * <td>PINOT_MODULE_SUBMODULE_ALERTS_EMAILADDRESS</td>
- * <td>Upper case format, which is recommended when using system environment variables.</td>
- * </tr>
  * </table>
  *
  */
 public class PinotConfiguration {
   public static final String CONFIG_PATHS_KEY = "config.paths";
+  public static final String ENV_DYNAMIC_CONFIG_KEY = "dynamic.env.config";
 
   private final CompositeConfiguration _configuration;
 
@@ -116,7 +120,7 @@ public class PinotConfiguration {
    * @param baseProperties to provide programmatically through a {@link Map}.
    */
   public PinotConfiguration(Map<String, Object> baseProperties) {
-    this(baseProperties, new HashMap<>());
+    this(baseProperties, new SystemEnvironment().getEnvironmentVariables());
   }
 
   /**
@@ -125,10 +129,12 @@ public class PinotConfiguration {
    * sanitized for relaxed binding. See {@link PinotConfiguration} for further details.
    *
    * @param baseProperties with highest precedences (e.g. CLI arguments)
-   * @param environmentVariables as a {@link Map}. Can be provided with {@link SystemEnvironment}
+   * @param environmentVariables as a {@link Map}.
    */
   public PinotConfiguration(Map<String, Object> baseProperties, Map<String, String> environmentVariables) {
-    _configuration = new CompositeConfiguration(computeConfigurationsFromSources(baseProperties, environmentVariables));
+    _configuration = new CompositeConfiguration(
+        applyDynamicEnvConfig(computeConfigurationsFromSources(baseProperties, environmentVariables),
+            environmentVariables));
   }
 
   /**
@@ -139,8 +145,8 @@ public class PinotConfiguration {
    */
   public PinotConfiguration(PinotFSSpec pinotFSSpec) {
     this(Optional.ofNullable(pinotFSSpec.getConfigs()).map(configs -> configs.entrySet().stream()
-        .collect(Collectors.<Entry<String, String>, String, Object>toMap(Entry::getKey, entry -> entry.getValue())))
-        .orElseGet(() -> new HashMap<>()));
+            .collect(Collectors.<Entry<String, String>, String, Object>toMap(Entry::getKey, Entry::getValue)))
+        .orElseGet(HashMap::new));
   }
 
   private static List<Configuration> computeConfigurationsFromSources(Configuration baseConfiguration,
@@ -148,22 +154,42 @@ public class PinotConfiguration {
     return computeConfigurationsFromSources(relaxConfigurationKeys(baseConfiguration), environmentVariables);
   }
 
+  /**
+   * This function templates the configuration from the env variables using env.dynamic.config to
+   * specify the mapping.
+   * E.g.
+   * env.dynamic.mapping=test.property
+   * test.property=ENV_VAR_NAME
+   * This function will look up `ENV_VAR_NAME` and insert its content in test.property.
+   *
+   * @param configurations List of configurations to template.
+   * @param environmentVariables Env used to fetch content to insert in the configuration.
+   * @return returns configuration
+   */
+  public static List<Configuration> applyDynamicEnvConfig(List<Configuration> configurations,
+      Map<String, String> environmentVariables) {
+    return configurations.stream().peek(configuration -> {
+      for (String dynamicEnvConfigVarName : configuration.getStringArray(ENV_DYNAMIC_CONFIG_KEY)) {
+        configuration.setProperty(dynamicEnvConfigVarName,
+            environmentVariables.get(configuration.getString(dynamicEnvConfigVarName)));
+      }
+    }).collect(Collectors.toList());
+  }
+
   private static List<Configuration> computeConfigurationsFromSources(Map<String, Object> baseProperties,
       Map<String, String> environmentVariables) {
     Map<String, Object> relaxedBaseProperties = relaxProperties(baseProperties);
+    // Env is only used to check for config paths to load.
     Map<String, String> relaxedEnvVariables = relaxEnvironmentVariables(environmentVariables);
 
-    Stream<Configuration> propertiesFromConfigPaths = Stream
-        .of(Optional.ofNullable(relaxedBaseProperties.get(CONFIG_PATHS_KEY)).map(Object::toString),
-            Optional.ofNullable(relaxedEnvVariables.get(CONFIG_PATHS_KEY)))
-
-        .filter(Optional::isPresent).map(Optional::get)
-
-        .flatMap(configPaths -> Arrays.stream(configPaths.split(",")))
-
-        .map(PinotConfiguration::loadProperties);
+    Stream<Configuration> propertiesFromConfigPaths =
+        Stream.of(Optional.ofNullable(relaxedBaseProperties.get(CONFIG_PATHS_KEY)).map(Object::toString),
+                Optional.ofNullable(relaxedEnvVariables.get(CONFIG_PATHS_KEY))).filter(Optional::isPresent)
+            .map(Optional::get).flatMap(configPaths -> Arrays.stream(configPaths.split(",")))
+            .map(PinotConfiguration::loadProperties);
 
-    return Stream.concat(Stream.of(relaxedBaseProperties, relaxedEnvVariables).map(e -> {
+    // Priority in CompositeConfiguration is CLI, ConfigFile(s)
+    return Stream.concat(Stream.of(relaxedBaseProperties).map(e -> {
       MapConfiguration mapConfiguration = new MapConfiguration(e);
       mapConfiguration.setListDelimiterHandler(new LegacyListDelimiterHandler(','));
       return mapConfiguration;
@@ -187,8 +213,7 @@ public class PinotConfiguration {
       PropertiesConfiguration propertiesConfiguration;
       if (configPath.startsWith("classpath:")) {
         propertiesConfiguration = CommonsConfigurationUtils.fromInputStream(
-            PinotConfiguration.class.getResourceAsStream(configPath.substring("classpath:".length())),
-            true, true);
+            PinotConfiguration.class.getResourceAsStream(configPath.substring("classpath:".length())), true, true);
       } else {
         propertiesConfiguration = CommonsConfigurationUtils.fromPath(configPath, true, true);
       }
@@ -201,16 +226,16 @@ public class PinotConfiguration {
   private static Map<String, Object> relaxConfigurationKeys(Configuration configuration) {
     return CommonsConfigurationUtils.getKeysStream(configuration).distinct()
 
-        .collect(Collectors.toMap(PinotConfiguration::relaxPropertyName, key -> configuration.getProperty(key)));
+        .collect(Collectors.toMap(PinotConfiguration::relaxPropertyName, configuration::getProperty));
   }
 
   private static Map<String, String> relaxEnvironmentVariables(Map<String, String> environmentVariables) {
-    return environmentVariables.entrySet().stream().filter(entry -> entry.getKey().startsWith("PINOT_"))
+    return environmentVariables.entrySet().stream()
         .collect(Collectors.toMap(PinotConfiguration::relaxEnvVarName, Entry::getValue));
   }
 
   private static String relaxEnvVarName(Entry<String, String> envVarEntry) {
-    return envVarEntry.getKey().substring(6).replace("_", ".").toLowerCase();
+    return envVarEntry.getKey().replace("_", ".").toLowerCase();
   }
 
   private static Map<String, Object> relaxProperties(Map<String, Object> properties) {
@@ -332,8 +357,8 @@ public class PinotConfiguration {
    * @return the property String value. Fallback to the provided default values if no property is found.
    */
   public List<String> getProperty(String name, List<String> defaultValues) {
-    return Optional
-        .of(Arrays.stream(_configuration.getStringArray(relaxPropertyName(name))).collect(Collectors.toList()))
+    return Optional.of(
+            Arrays.stream(_configuration.getStringArray(relaxPropertyName(name))).collect(Collectors.toList()))
         .filter(list -> !list.isEmpty()).orElse(defaultValues);
   }
 
diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/env/PinotConfigurationTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/env/PinotConfigurationTest.java
index 8ae846099c..a7c1f0783e 100644
--- a/pinot-spi/src/test/java/org/apache/pinot/spi/env/PinotConfigurationTest.java
+++ b/pinot-spi/src/test/java/org/apache/pinot/spi/env/PinotConfigurationTest.java
@@ -83,7 +83,7 @@ public class PinotConfigurationTest {
     properties.put("property.3.key", "val1");
     properties.put("property.4.key", "val1");
 
-    PinotConfiguration pinotConfiguration = new PinotConfiguration(properties);
+    PinotConfiguration pinotConfiguration = new PinotConfiguration(properties, new HashMap<>());
 
     List<String> keys = pinotConfiguration.getKeys();
     Assert.assertTrue(keys.contains("property.1.key"));
@@ -133,10 +133,7 @@ public class PinotConfigurationTest {
 
     baseProperties.put("controller.host", "cli-argument-controller-host");
     baseProperties.put("config.paths", "classpath:/pinot-configuration-1.properties");
-    mockedEnvironmentVariables.put("PINOT_CONTROLLER_HOST", "env-var-controller-host");
-    mockedEnvironmentVariables.put("PINOT_CONTROLLER_PORT", "env-var-controller-port");
-    mockedEnvironmentVariables.put("PINOT_RELAXEDPROPERTY_TEST", "true");
-    mockedEnvironmentVariables.put("PINOT_CONFIG_PATHS", configFile2 + "," + configFile3);
+    mockedEnvironmentVariables.put("CONFIG_PATHS", configFile2 + "," + configFile3);
 
     copyClasspathResource("/pinot-configuration-2.properties", configFile2);
     copyClasspathResource("/pinot-configuration-3.properties", configFile3);
@@ -146,9 +143,6 @@ public class PinotConfigurationTest {
     // Tests that cli arguments have the highest priority.
     Assert.assertEquals(configuration.getProperty("controller.host"), "cli-argument-controller-host");
 
-    // Tests that environment variable have priority overs config.paths properties.
-    Assert.assertEquals(configuration.getProperty("controller.port"), "env-var-controller-port");
-
     // Tests that config.paths properties provided through cli arguments are prioritized.
     Assert.assertEquals(configuration.getProperty("controller.cluster-name"), "config-path-1-cluster-name");
 
@@ -162,9 +156,35 @@ public class PinotConfigurationTest {
     // Tests properties provided through the last config file of a config.paths array.
     Assert.assertEquals(configuration.getProperty("controller.config-paths-multi-value-test-2"),
         "config-path-3-config-paths-multi-value-test-2");
+  }
+
+  @Test
+  public void assertDynamicEnvConfig()
+      throws IOException {
+    Map<String, Object> baseProperties = new HashMap<>();
+    Map<String, String> mockedEnvironmentVariables = new HashMap<>();
 
-    // Tests relaxed binding on environment variables
-    Assert.assertEquals(configuration.getProperty("relaxed-property.test"), "true");
+    String configFile = File.createTempFile("pinot-configuration-test-4", ".properties").getAbsolutePath();
+
+    baseProperties.put("server.host", "ENV_SERVER_HOST");
+    baseProperties.put("not.templated.cli", "static-value");
+    baseProperties.put("dynamic.env.config", "server.host");
+
+    mockedEnvironmentVariables.put("ENV_VAR_HOST", "test-host");
+    mockedEnvironmentVariables.put("TEST_PROPERTY", "test-property");
+    mockedEnvironmentVariables.put("ENV_SERVER_HOST", "test-server-host");
+
+    baseProperties.put("config.paths", "classpath:/pinot-configuration-4.properties");
+    copyClasspathResource("/pinot-configuration-4.properties", configFile);
+
+    PinotConfiguration configuration = new PinotConfiguration(baseProperties, mockedEnvironmentVariables);
+
+    // Tests that cli arguments have the highest priority.
+    Assert.assertEquals(configuration.getProperty("server.host"), "test-server-host");
+    // Checking for non templated values
+    Assert.assertEquals(configuration.getProperty("not.templated.cli"), "static-value");
+    // Templated values in configFile
+    Assert.assertEquals(configuration.getProperty("pinot.controller.host"), "test-host");
   }
 
   @Test(expectedExceptions = IllegalArgumentException.class)
@@ -180,8 +200,8 @@ public class PinotConfigurationTest {
   public void assertPropertiesFromBaseConfiguration()
       throws ConfigurationException {
     PropertiesConfiguration propertiesConfiguration = CommonsConfigurationUtils.fromPath(
-        PropertiesConfiguration.class.getClassLoader().getResource("pinot-configuration-1.properties").getFile(),
-        true, true);
+        PropertiesConfiguration.class.getClassLoader().getResource("pinot-configuration-1.properties").getFile(), true,
+        true);
 
     PinotConfiguration config = new PinotConfiguration(propertiesConfiguration);
 
diff --git a/pinot-spi/src/test/resources/pinot-configuration-4.properties b/pinot-spi/src/test/resources/pinot-configuration-4.properties
new file mode 100644
index 0000000000..b8c5913e75
--- /dev/null
+++ b/pinot-spi/src/test/resources/pinot-configuration-4.properties
@@ -0,0 +1,23 @@
+#
+# 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.
+#
+
+dynamic.env.config=pinot.controller.host,pinot.test.property,server.host
+
+pinot.controller.host=ENV_VAR_HOST
+server.host=incorrect-value


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org