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 2022/04/07 03:44:09 UTC

[GitHub] [flink] HuangXingBo commented on a diff in pull request #19333: [FLINK-26190][python] Remove getTableConfig from ExecNodeConfiguration

HuangXingBo commented on code in PR #19333:
URL: https://github.com/apache/flink/pull/19333#discussion_r844600618


##########
flink-python/src/main/java/org/apache/flink/python/PythonConfig.java:
##########
@@ -19,152 +19,112 @@
 package org.apache.flink.python;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigUtils;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.python.util.PythonDependencyUtils;
+import org.apache.flink.table.api.config.TableConfigOptions;
+import org.apache.flink.util.Preconditions;
 
-import javax.annotation.Nullable;
-
-import java.io.Serializable;
+import java.time.ZoneId;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 
+import static org.apache.flink.python.PythonOptions.PYTHON_LOOPBACK_SERVER_ADDRESS;
+
 /** Configurations for the Python job which are used at run time. */
 @Internal
-public class PythonConfig implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-
-    /** Max number of elements to include in a bundle. */
-    private final int maxBundleSize;
-
-    /** Max duration of a bundle. */
-    private final long maxBundleTimeMills;
-
-    /** Max number of elements to include in an arrow batch. */
-    private final int maxArrowBatchSize;
-
-    /**
-     * The python files uploaded by pyflink.table.TableEnvironment#add_python_file() or command line
-     * option "-pyfs". The key is the file key in distribute cache and the value is the
-     * corresponding origin file name.
-     */
-    private final Map<String, String> pythonFilesInfo;
+public class PythonConfig implements ReadableConfig {
 
-    /**
-     * The file key of the requirements file in distribute cache. It is specified by
-     * pyflink.table.TableEnvironment#set_python_requirements() or command line option "-pyreq".
-     */
-    @Nullable private final String pythonRequirementsFileInfo;
+    private static final List<ConfigOption<?>> PYTHON_CONFIG_OPTIONS;
 
-    /**
-     * The file key of the requirements cached directory in distribute cache. It is specified by
-     * pyflink.table.TableEnvironment#set_python_requirements() or command line option "-pyreq". It
-     * is used to support installing python packages offline.
-     */
-    @Nullable private final String pythonRequirementsCacheDirInfo;
+    static {
+        PYTHON_CONFIG_OPTIONS =
+                new ArrayList<>(ConfigUtils.getAllConfigOptions(PythonOptions.class));
+    }
 
     /**
-     * The python archives uploaded by pyflink.table.TableEnvironment#add_python_archive() or
-     * command line option "-pyarch". The key is the file key of the archives in distribute cache
-     * and the value is the name of the directory to extract to.
+     * Configuration adopted from the outer layer, e.g. flink-conf.yaml, command line arguments,
+     * TableConfig, etc.
      */
-    private final Map<String, String> pythonArchivesInfo;
+    private final ReadableConfig configuration;
 
     /**
-     * The path of the python interpreter (e.g. /usr/local/bin/python) specified by
-     * pyflink.table.TableConfig#set_python_executable() or command line option "-pyexec".
+     * Configuration generated in the dependency management mechanisms. See {@link
+     * PythonDependencyUtils.PythonDependencyManager} for more details.
      */
-    private final String pythonExec;
-
-    /** Whether metric is enabled. */
-    private final boolean metricEnabled;
-
-    /** Whether to use managed memory for the Python worker. */
-    private final boolean isUsingManagedMemory;
-
-    /** The Configuration that contains execution configs and dependencies info. */
-    private final Configuration config;
-
-    /** Whether profile is enabled. */
-    private final boolean profileEnabled;
-
-    /** Execution Mode. */
-    private final String executionMode;
-
-    public PythonConfig(Configuration config) {
-        this.config = config;
-        maxBundleSize = config.get(PythonOptions.MAX_BUNDLE_SIZE);
-        maxBundleTimeMills = config.get(PythonOptions.MAX_BUNDLE_TIME_MILLS);
-        maxArrowBatchSize = config.get(PythonOptions.MAX_ARROW_BATCH_SIZE);
-        pythonFilesInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_FILES).orElse(new HashMap<>());
-        pythonRequirementsFileInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE)
-                        .orElse(new HashMap<>())
-                        .get(PythonDependencyUtils.FILE);
-        pythonRequirementsCacheDirInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE)
-                        .orElse(new HashMap<>())
-                        .get(PythonDependencyUtils.CACHE);
-        pythonArchivesInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_ARCHIVES).orElse(new HashMap<>());
-        pythonExec = config.get(PythonOptions.PYTHON_EXECUTABLE);
-        metricEnabled = config.getBoolean(PythonOptions.PYTHON_METRIC_ENABLED);
-        isUsingManagedMemory = config.getBoolean(PythonOptions.USE_MANAGED_MEMORY);
-        profileEnabled = config.getBoolean(PythonOptions.PYTHON_PROFILE_ENABLED);
-        executionMode = config.getString(PythonOptions.PYTHON_EXECUTION_MODE);
-    }
-
-    public int getMaxBundleSize() {
-        return maxBundleSize;
-    }
-
-    public long getMaxBundleTimeMills() {
-        return maxBundleTimeMills;
-    }
-
-    public int getMaxArrowBatchSize() {
-        return maxArrowBatchSize;
-    }
-
-    public Map<String, String> getPythonFilesInfo() {
-        return pythonFilesInfo;
-    }
-
-    public Optional<String> getPythonRequirementsFileInfo() {
-        return Optional.ofNullable(pythonRequirementsFileInfo);
-    }
-
-    public Optional<String> getPythonRequirementsCacheDirInfo() {
-        return Optional.ofNullable(pythonRequirementsCacheDirInfo);
-    }
+    private final ReadableConfig pythonDependencyConfiguration;
 
-    public Map<String, String> getPythonArchivesInfo() {
-        return pythonArchivesInfo;
+    public PythonConfig(ReadableConfig configuration) {
+        this.configuration = Preconditions.checkNotNull(configuration);
+        this.pythonDependencyConfiguration = new Configuration();
     }
 
-    public String getPythonExec() {
-        return pythonExec;
+    public PythonConfig(
+            ReadableConfig configuration, ReadableConfig pythonDependencyConfiguration) {
+        this.configuration = Preconditions.checkNotNull(configuration);
+        this.pythonDependencyConfiguration =
+                Preconditions.checkNotNull(pythonDependencyConfiguration);
     }
 
-    public String getExecutionMode() {
-        return executionMode;
+    @Override
+    public <T> T get(ConfigOption<T> option) {
+        return pythonDependencyConfiguration
+                .getOptional(option)
+                .orElseGet(() -> configuration.get(option));
     }
 
-    public boolean isMetricEnabled() {
-        return metricEnabled;
+    @Override
+    public <T> Optional<T> getOptional(ConfigOption<T> option) {
+        final Optional<T> value = pythonDependencyConfiguration.getOptional(option);
+        if (value.isPresent()) {
+            return value;
+        }
+        return configuration.getOptional(option);
     }
 
-    public boolean isProfileEnabled() {
-        return profileEnabled;
-    }
+    public Configuration toConfiguration() {
+        final Configuration config = new Configuration();
+        PYTHON_CONFIG_OPTIONS.forEach(
+                option ->
+                        getOptional((ConfigOption) option)
+                                .ifPresent(v -> config.set((ConfigOption) option, v)));
+
+        // prepare the job options
+        Map<String, String> jobOptions = config.get(PythonOptions.PYTHON_JOB_OPTIONS);
+        if (jobOptions == null) {
+            jobOptions = new HashMap<>();
+            config.set(PythonOptions.PYTHON_JOB_OPTIONS, jobOptions);
+        }
+        jobOptions.put(
+                PythonOptions.STATE_CACHE_SIZE.key(),
+                String.valueOf(config.get(PythonOptions.STATE_CACHE_SIZE)));
+        jobOptions.put(
+                PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE.key(),
+                String.valueOf(config.get(PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE)));
+        jobOptions.put("table.exec.timezone", getLocalTimeZone(configuration).getId());

Review Comment:
   Would it be better to use the key of TableConfigOptions.LOCAL_TIME_ZONE directly instead of adding a new table.exec.timezone?
   



##########
flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java:
##########
@@ -179,14 +171,16 @@ private PythonDependencyManager(
          *
          * @param filePath The path of the Python dependency.
          */
-        private void addPythonFile(String filePath) {
+        private void addPythonFile(Configuration toConfiguration, String filePath) {

Review Comment:
   What is the meaning of variable name `toConfiguration`



##########
flink-python/src/main/java/org/apache/flink/python/PythonConfig.java:
##########
@@ -19,152 +19,112 @@
 package org.apache.flink.python;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigUtils;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.python.util.PythonDependencyUtils;
+import org.apache.flink.table.api.config.TableConfigOptions;
+import org.apache.flink.util.Preconditions;
 
-import javax.annotation.Nullable;
-
-import java.io.Serializable;
+import java.time.ZoneId;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 
+import static org.apache.flink.python.PythonOptions.PYTHON_LOOPBACK_SERVER_ADDRESS;
+
 /** Configurations for the Python job which are used at run time. */
 @Internal
-public class PythonConfig implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-
-    /** Max number of elements to include in a bundle. */
-    private final int maxBundleSize;
-
-    /** Max duration of a bundle. */
-    private final long maxBundleTimeMills;
-
-    /** Max number of elements to include in an arrow batch. */
-    private final int maxArrowBatchSize;
-
-    /**
-     * The python files uploaded by pyflink.table.TableEnvironment#add_python_file() or command line
-     * option "-pyfs". The key is the file key in distribute cache and the value is the
-     * corresponding origin file name.
-     */
-    private final Map<String, String> pythonFilesInfo;
+public class PythonConfig implements ReadableConfig {
 
-    /**
-     * The file key of the requirements file in distribute cache. It is specified by
-     * pyflink.table.TableEnvironment#set_python_requirements() or command line option "-pyreq".
-     */
-    @Nullable private final String pythonRequirementsFileInfo;
+    private static final List<ConfigOption<?>> PYTHON_CONFIG_OPTIONS;
 
-    /**
-     * The file key of the requirements cached directory in distribute cache. It is specified by
-     * pyflink.table.TableEnvironment#set_python_requirements() or command line option "-pyreq". It
-     * is used to support installing python packages offline.
-     */
-    @Nullable private final String pythonRequirementsCacheDirInfo;
+    static {
+        PYTHON_CONFIG_OPTIONS =
+                new ArrayList<>(ConfigUtils.getAllConfigOptions(PythonOptions.class));
+    }
 
     /**
-     * The python archives uploaded by pyflink.table.TableEnvironment#add_python_archive() or
-     * command line option "-pyarch". The key is the file key of the archives in distribute cache
-     * and the value is the name of the directory to extract to.
+     * Configuration adopted from the outer layer, e.g. flink-conf.yaml, command line arguments,
+     * TableConfig, etc.
      */
-    private final Map<String, String> pythonArchivesInfo;
+    private final ReadableConfig configuration;
 
     /**
-     * The path of the python interpreter (e.g. /usr/local/bin/python) specified by
-     * pyflink.table.TableConfig#set_python_executable() or command line option "-pyexec".
+     * Configuration generated in the dependency management mechanisms. See {@link
+     * PythonDependencyUtils.PythonDependencyManager} for more details.
      */
-    private final String pythonExec;
-
-    /** Whether metric is enabled. */
-    private final boolean metricEnabled;
-
-    /** Whether to use managed memory for the Python worker. */
-    private final boolean isUsingManagedMemory;
-
-    /** The Configuration that contains execution configs and dependencies info. */
-    private final Configuration config;
-
-    /** Whether profile is enabled. */
-    private final boolean profileEnabled;
-
-    /** Execution Mode. */
-    private final String executionMode;
-
-    public PythonConfig(Configuration config) {
-        this.config = config;
-        maxBundleSize = config.get(PythonOptions.MAX_BUNDLE_SIZE);
-        maxBundleTimeMills = config.get(PythonOptions.MAX_BUNDLE_TIME_MILLS);
-        maxArrowBatchSize = config.get(PythonOptions.MAX_ARROW_BATCH_SIZE);
-        pythonFilesInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_FILES).orElse(new HashMap<>());
-        pythonRequirementsFileInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE)
-                        .orElse(new HashMap<>())
-                        .get(PythonDependencyUtils.FILE);
-        pythonRequirementsCacheDirInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE)
-                        .orElse(new HashMap<>())
-                        .get(PythonDependencyUtils.CACHE);
-        pythonArchivesInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_ARCHIVES).orElse(new HashMap<>());
-        pythonExec = config.get(PythonOptions.PYTHON_EXECUTABLE);
-        metricEnabled = config.getBoolean(PythonOptions.PYTHON_METRIC_ENABLED);
-        isUsingManagedMemory = config.getBoolean(PythonOptions.USE_MANAGED_MEMORY);
-        profileEnabled = config.getBoolean(PythonOptions.PYTHON_PROFILE_ENABLED);
-        executionMode = config.getString(PythonOptions.PYTHON_EXECUTION_MODE);
-    }
-
-    public int getMaxBundleSize() {
-        return maxBundleSize;
-    }
-
-    public long getMaxBundleTimeMills() {
-        return maxBundleTimeMills;
-    }
-
-    public int getMaxArrowBatchSize() {
-        return maxArrowBatchSize;
-    }
-
-    public Map<String, String> getPythonFilesInfo() {
-        return pythonFilesInfo;
-    }
-
-    public Optional<String> getPythonRequirementsFileInfo() {
-        return Optional.ofNullable(pythonRequirementsFileInfo);
-    }
-
-    public Optional<String> getPythonRequirementsCacheDirInfo() {
-        return Optional.ofNullable(pythonRequirementsCacheDirInfo);
-    }
+    private final ReadableConfig pythonDependencyConfiguration;
 
-    public Map<String, String> getPythonArchivesInfo() {
-        return pythonArchivesInfo;
+    public PythonConfig(ReadableConfig configuration) {
+        this.configuration = Preconditions.checkNotNull(configuration);
+        this.pythonDependencyConfiguration = new Configuration();
     }
 
-    public String getPythonExec() {
-        return pythonExec;
+    public PythonConfig(
+            ReadableConfig configuration, ReadableConfig pythonDependencyConfiguration) {
+        this.configuration = Preconditions.checkNotNull(configuration);
+        this.pythonDependencyConfiguration =
+                Preconditions.checkNotNull(pythonDependencyConfiguration);
     }
 
-    public String getExecutionMode() {
-        return executionMode;
+    @Override
+    public <T> T get(ConfigOption<T> option) {
+        return pythonDependencyConfiguration
+                .getOptional(option)
+                .orElseGet(() -> configuration.get(option));
     }
 
-    public boolean isMetricEnabled() {
-        return metricEnabled;
+    @Override
+    public <T> Optional<T> getOptional(ConfigOption<T> option) {
+        final Optional<T> value = pythonDependencyConfiguration.getOptional(option);
+        if (value.isPresent()) {
+            return value;
+        }
+        return configuration.getOptional(option);
     }
 
-    public boolean isProfileEnabled() {
-        return profileEnabled;
-    }
+    public Configuration toConfiguration() {
+        final Configuration config = new Configuration();
+        PYTHON_CONFIG_OPTIONS.forEach(
+                option ->
+                        getOptional((ConfigOption) option)
+                                .ifPresent(v -> config.set((ConfigOption) option, v)));
+
+        // prepare the job options
+        Map<String, String> jobOptions = config.get(PythonOptions.PYTHON_JOB_OPTIONS);
+        if (jobOptions == null) {
+            jobOptions = new HashMap<>();
+            config.set(PythonOptions.PYTHON_JOB_OPTIONS, jobOptions);
+        }
+        jobOptions.put(
+                PythonOptions.STATE_CACHE_SIZE.key(),
+                String.valueOf(config.get(PythonOptions.STATE_CACHE_SIZE)));
+        jobOptions.put(
+                PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE.key(),
+                String.valueOf(config.get(PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE)));
+        jobOptions.put("table.exec.timezone", getLocalTimeZone(configuration).getId());
+        if (config.contains(PYTHON_LOOPBACK_SERVER_ADDRESS)) {

Review Comment:
   ditto. Use the key of `PYTHON_LOOPBACK_SERVER_ADDRESS`?



##########
flink-python/src/main/java/org/apache/flink/python/PythonConfig.java:
##########
@@ -19,152 +19,112 @@
 package org.apache.flink.python;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigUtils;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.python.util.PythonDependencyUtils;
+import org.apache.flink.table.api.config.TableConfigOptions;
+import org.apache.flink.util.Preconditions;
 
-import javax.annotation.Nullable;
-
-import java.io.Serializable;
+import java.time.ZoneId;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 
+import static org.apache.flink.python.PythonOptions.PYTHON_LOOPBACK_SERVER_ADDRESS;
+
 /** Configurations for the Python job which are used at run time. */
 @Internal
-public class PythonConfig implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-
-    /** Max number of elements to include in a bundle. */
-    private final int maxBundleSize;
-
-    /** Max duration of a bundle. */
-    private final long maxBundleTimeMills;
-
-    /** Max number of elements to include in an arrow batch. */
-    private final int maxArrowBatchSize;
-
-    /**
-     * The python files uploaded by pyflink.table.TableEnvironment#add_python_file() or command line
-     * option "-pyfs". The key is the file key in distribute cache and the value is the
-     * corresponding origin file name.
-     */
-    private final Map<String, String> pythonFilesInfo;
+public class PythonConfig implements ReadableConfig {
 
-    /**
-     * The file key of the requirements file in distribute cache. It is specified by
-     * pyflink.table.TableEnvironment#set_python_requirements() or command line option "-pyreq".
-     */
-    @Nullable private final String pythonRequirementsFileInfo;
+    private static final List<ConfigOption<?>> PYTHON_CONFIG_OPTIONS;
 
-    /**
-     * The file key of the requirements cached directory in distribute cache. It is specified by
-     * pyflink.table.TableEnvironment#set_python_requirements() or command line option "-pyreq". It
-     * is used to support installing python packages offline.
-     */
-    @Nullable private final String pythonRequirementsCacheDirInfo;
+    static {
+        PYTHON_CONFIG_OPTIONS =
+                new ArrayList<>(ConfigUtils.getAllConfigOptions(PythonOptions.class));
+    }
 
     /**
-     * The python archives uploaded by pyflink.table.TableEnvironment#add_python_archive() or
-     * command line option "-pyarch". The key is the file key of the archives in distribute cache
-     * and the value is the name of the directory to extract to.
+     * Configuration adopted from the outer layer, e.g. flink-conf.yaml, command line arguments,
+     * TableConfig, etc.
      */
-    private final Map<String, String> pythonArchivesInfo;
+    private final ReadableConfig configuration;
 
     /**
-     * The path of the python interpreter (e.g. /usr/local/bin/python) specified by
-     * pyflink.table.TableConfig#set_python_executable() or command line option "-pyexec".
+     * Configuration generated in the dependency management mechanisms. See {@link
+     * PythonDependencyUtils.PythonDependencyManager} for more details.
      */
-    private final String pythonExec;
-
-    /** Whether metric is enabled. */
-    private final boolean metricEnabled;
-
-    /** Whether to use managed memory for the Python worker. */
-    private final boolean isUsingManagedMemory;
-
-    /** The Configuration that contains execution configs and dependencies info. */
-    private final Configuration config;
-
-    /** Whether profile is enabled. */
-    private final boolean profileEnabled;
-
-    /** Execution Mode. */
-    private final String executionMode;
-
-    public PythonConfig(Configuration config) {
-        this.config = config;
-        maxBundleSize = config.get(PythonOptions.MAX_BUNDLE_SIZE);
-        maxBundleTimeMills = config.get(PythonOptions.MAX_BUNDLE_TIME_MILLS);
-        maxArrowBatchSize = config.get(PythonOptions.MAX_ARROW_BATCH_SIZE);
-        pythonFilesInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_FILES).orElse(new HashMap<>());
-        pythonRequirementsFileInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE)
-                        .orElse(new HashMap<>())
-                        .get(PythonDependencyUtils.FILE);
-        pythonRequirementsCacheDirInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE)
-                        .orElse(new HashMap<>())
-                        .get(PythonDependencyUtils.CACHE);
-        pythonArchivesInfo =
-                config.getOptional(PythonDependencyUtils.PYTHON_ARCHIVES).orElse(new HashMap<>());
-        pythonExec = config.get(PythonOptions.PYTHON_EXECUTABLE);
-        metricEnabled = config.getBoolean(PythonOptions.PYTHON_METRIC_ENABLED);
-        isUsingManagedMemory = config.getBoolean(PythonOptions.USE_MANAGED_MEMORY);
-        profileEnabled = config.getBoolean(PythonOptions.PYTHON_PROFILE_ENABLED);
-        executionMode = config.getString(PythonOptions.PYTHON_EXECUTION_MODE);
-    }
-
-    public int getMaxBundleSize() {
-        return maxBundleSize;
-    }
-
-    public long getMaxBundleTimeMills() {
-        return maxBundleTimeMills;
-    }
-
-    public int getMaxArrowBatchSize() {
-        return maxArrowBatchSize;
-    }
-
-    public Map<String, String> getPythonFilesInfo() {
-        return pythonFilesInfo;
-    }
-
-    public Optional<String> getPythonRequirementsFileInfo() {
-        return Optional.ofNullable(pythonRequirementsFileInfo);
-    }
-
-    public Optional<String> getPythonRequirementsCacheDirInfo() {
-        return Optional.ofNullable(pythonRequirementsCacheDirInfo);
-    }
+    private final ReadableConfig pythonDependencyConfiguration;
 
-    public Map<String, String> getPythonArchivesInfo() {
-        return pythonArchivesInfo;
+    public PythonConfig(ReadableConfig configuration) {

Review Comment:
   I don't find where this constructor method is used



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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org