You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/07/01 13:28:04 UTC

[GitHub] [incubator-pinot] daniellavoie commented on a change in pull request #5608: Pinot Configuration Refactoring - Phase 1

daniellavoie commented on a change in pull request #5608:
URL: https://github.com/apache/incubator-pinot/pull/5608#discussion_r448362557



##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/env/PinotConfiguration.java
##########
@@ -0,0 +1,250 @@
+/**
+ * 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.pinot.spi.env;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.ConfigurationUtils;
+import org.apache.commons.configuration.MapConfiguration;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.pinot.spi.ingestion.batch.spec.PinotFSSpec;
+
+
+public class PinotConfiguration {
+  public static final String CONFIG_PATHS_KEY = "config.paths";
+
+  private final CompositeConfiguration configuration;
+
+  public PinotConfiguration() {
+    this(new HashMap<>());
+  }
+
+  public PinotConfiguration(Configuration baseConfiguration) {
+    this.configuration =
+        new CompositeConfiguration(computeConfigurationsFromSources(baseConfiguration, new HashMap<>()));
+  }
+
+  public PinotConfiguration(Map<String, Object> baseProperties) {
+    this(baseProperties, new HashMap<>());
+  }
+
+  public PinotConfiguration(Map<String, Object> baseProperties, Map<String, String> environmentVariables) {
+    this.configuration =
+        new CompositeConfiguration(computeConfigurationsFromSources(baseProperties, environmentVariables));
+  }
+
+  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<>()));
+  }
+
+  /**
+   * Overwrites a property value in memory.
+   * 
+   * @param name of the property to append in memory. Applies relaxed binding on the property name.
+   * @param value to overwrite in memory
+   * 
+   * @deprecated Configurations should be immutable. Prefer creating a new {@link #PinotConfiguration} with base properties to overwrite properties.
+   */
+  public void addProperty(String name, Object value) {
+    configuration.addProperty(relaxPropertyName(name), value);
+  }
+
+  public PinotConfiguration clone() {
+    return new PinotConfiguration(ConfigurationUtils.cloneConfiguration(configuration));
+  }
+
+  private static List<Configuration> computeConfigurationsFromSources(Configuration baseConfiguration,
+      Map<String, String> environmentVariables) {
+    return computeConfigurationsFromSources(relaxConfigurationKeys(baseConfiguration), environmentVariables);
+  }
+
+  private static List<Configuration> computeConfigurationsFromSources(Map<String, Object> baseProperties,
+      Map<String, String> environmentVariables) {
+    Map<String, Object> relaxedBaseProperties = relaxProperties(baseProperties);
+    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);
+
+    return Stream.concat(Stream.of(relaxedBaseProperties, relaxedEnvVariables).map(MapConfiguration::new),
+        propertiesFromConfigPaths).collect(Collectors.toList());
+  }
+
+  public boolean containsKey(String key) {
+    return configuration.containsKey(relaxPropertyName(key));
+  }
+
+  public List<String> getKeys() {
+    return CommonsConfigurationUtils.getKeys(configuration);
+  }
+
+  public String getProperty(String name) {
+    return getProperty(name, configuration);
+  }
+
+  private static String getProperty(String name, Configuration configuration) {
+    return Optional.of(configuration.getStringArray(relaxPropertyName(name)))
+
+        .filter(values -> values.length > 0)
+
+        .map(Arrays::stream)
+
+        .map(stream -> stream.collect(Collectors.joining(",")))
+
+        .orElse(null);
+  }
+
+  public <T> T getProperty(String name, Class<T> returnType) {
+    return getProperty(name, null, returnType);
+  }
+
+  public List<String> getProperty(String name, List<String> defaultValue) {
+    return Optional
+        .of(Arrays.stream(configuration.getStringArray(relaxPropertyName(name))).collect(Collectors.toList()))
+        .filter(list -> !list.isEmpty()).orElse(defaultValue);
+  }
+
+  public Object getRawProperty(String name) {
+    return getRawProperty(name, null);
+  }
+
+  public Object getRawProperty(String name, Object defaultValue) {
+    String relaxedPropertyName = relaxPropertyName(name);
+    if (!configuration.containsKey(relaxedPropertyName)) {
+      return defaultValue;
+    }
+
+    return configuration.getProperty(relaxedPropertyName);
+  }
+
+  public String getProperty(String name, String defaultValue) {
+    return getRawProperty(name, defaultValue).toString();
+  }
+
+  private <T> T getProperty(String name, T defaultValue, Class<T> returnType) {
+    String relaxedPropertyName = relaxPropertyName(name);
+    if (!configuration.containsKey(relaxedPropertyName)) {
+      return defaultValue;
+    }
+
+    return PropertyConverter.convert(getRawProperty(name, defaultValue), returnType);
+  }
+
+  public boolean getProperty(String name, boolean defaultValue) {
+    return getProperty(name, defaultValue, Boolean.class);
+  }
+
+  public int getProperty(String name, int defaultValue) {
+    return getProperty(name, defaultValue, Integer.class);
+  }
+
+  public long getProperty(String name, long defaultValue) {
+    return getProperty(name, defaultValue, Long.class);
+  }
+
+  public double getProperty(String name, double defaultValue) {
+    return getProperty(name, defaultValue, Double.class);
+  }
+
+  private static Configuration loadProperties(String configPath) {
+    try {
+      PropertiesConfiguration propertiesConfiguration = new PropertiesConfiguration();
+
+      propertiesConfiguration.setIOFactory(new ConfigFilePropertyReaderFactory());
+      if (configPath.startsWith("classpath:")) {
+        propertiesConfiguration
+            .load(PinotConfiguration.class.getResourceAsStream(configPath.substring("classpath:".length())));
+      } else {
+        propertiesConfiguration.load(configPath);
+      }
+
+      return propertiesConfiguration;
+    } catch (ConfigurationException e) {
+      throw new RuntimeException("Could not read properties from " + configPath, e);
+    }
+  }
+
+  private static Map<String, Object> relaxConfigurationKeys(Configuration configuration) {
+    return CommonsConfigurationUtils.getKeysStream(configuration)
+
+        .collect(Collectors.toMap(PinotConfiguration::relaxPropertyName, key -> getProperty(key, configuration)));
+  }
+
+  private static Map<String, String> relaxEnvironmentVariables(Map<String, String> environmentVariables) {
+    return environmentVariables.entrySet().stream().filter(entry -> entry.getKey().startsWith("PINOT_"))
+        .collect(Collectors.toMap(PinotConfiguration::relaxEnvVarName, Entry::getValue));
+  }
+
+  private static String relaxEnvVarName(Entry<String, String> envVarEntry) {
+    return envVarEntry.getKey().substring(6).replace("_", ".").toLowerCase();
+  }
+
+  private static Map<String, Object> relaxProperties(Map<String, Object> properties) {
+    return properties.entrySet().stream()
+        .collect(Collectors.toMap(PinotConfiguration::relaxPropertyName, Entry::getValue));
+  }
+
+  private static String relaxPropertyName(Entry<String, Object> propertyEntry) {
+    return relaxPropertyName(propertyEntry.getKey());
+  }
+
+  private static String relaxPropertyName(String propertyName) {
+    return propertyName.replace("-", "").toLowerCase();

Review comment:
       This is an intentional behaviour. To support properties provided through environment variables, the relaxed binding mechanism sanitize all properties inserted and extracted from `PinotConfiguration`. There is no breaking change since a value configured with `controller.upload.onlineToOfflineTimeout` will be resolvable with the any of the following configs key: 
   
   * `controller.upload.onlineToOfflineTimeout`
   * `controller.upload.onlinetoofflinetimeout`
   * `controller.upload.online-to-offline-timeout`
   * `PINOT_CONTROLLER_UPLOAD_ONLINETOOFFLINETIMEOUT`




----------------------------------------------------------------
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



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