igalshilman commented on a change in pull request #27: [FLINK-16149][core] Set 
configurations using StreamExecutionEnvironment#getConfiguration
URL: https://github.com/apache/flink-statefun/pull/27#discussion_r381951960
 
 

 ##########
 File path: 
statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/StatefulFunctionsJob.java
 ##########
 @@ -17,59 +17,62 @@
  */
 package org.apache.flink.statefun.flink.core;
 
+import java.io.File;
 import java.net.URL;
 import java.net.URLClassLoader;
+import java.util.Map;
 import java.util.Objects;
 import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
 import 
org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders;
-import org.apache.flink.statefun.flink.core.common.ConfigurationUtil;
 import org.apache.flink.statefun.flink.core.translation.FlinkUniverse;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 
+@SuppressWarnings("JavaReflectionMemberAccess")
 public class StatefulFunctionsJob {
 
+  private static final String CONFIG_DIRECTORY_FALLBACK_1 = "../conf";
+  private static final String CONFIG_DIRECTORY_FALLBACK_2 = "conf";
+
   public static void main(String... args) throws Exception {
     ParameterTool parameterTool = ParameterTool.fromArgs(args);
-    Configuration configuration = parameterTool.getConfiguration();
+    Map<String, String> globalConfigurations = parameterTool.toMap();
+
+    String configDirectory = getConfigurationDirectoryFromEnv();
+    Configuration flinkConf = 
GlobalConfiguration.loadConfiguration(configDirectory);
+    StatefulFunctionsConfig stateFunConfig = new 
StatefulFunctionsConfig(flinkConf);
+    stateFunConfig.setGlobalConfigurations(globalConfigurations);
+    stateFunConfig.setProvider(new 
StatefulFunctionsUniverses.ClassPathUniverseProvider());
 
-    main(configuration);
+    main(stateFunConfig, new Configuration());
   }
 
-  public static void main(Configuration configuration) throws Exception {
-    Objects.requireNonNull(configuration);
+  public static void main(StatefulFunctionsConfig stateFunConfig, 
Configuration flinkConf)
+      throws Exception {
+    Objects.requireNonNull(stateFunConfig);
+    Objects.requireNonNull(flinkConf);
 
     setDefaultContextClassLoaderIfAbsent();
-    setDefaultProviderIfAbsent(
-        configuration, new 
StatefulFunctionsUniverses.ClassPathUniverseProvider());
+
+    StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+    env.configure(flinkConf, Thread.currentThread().getContextClassLoader());
+
+    env.getConfig().enableObjectReuse();
 
     final StatefulFunctionsUniverse statefulFunctionsUniverse =
         StatefulFunctionsUniverses.get(
-            Thread.currentThread().getContextClassLoader(), configuration);
+            Thread.currentThread().getContextClassLoader(), stateFunConfig);
 
     final StatefulFunctionsUniverseValidator 
statefulFunctionsUniverseValidator =
         new StatefulFunctionsUniverseValidator();
     statefulFunctionsUniverseValidator.validate(statefulFunctionsUniverse);
 
-    StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
-    setDefaultConfiguration(configuration, env);
-
-    FlinkUniverse flinkUniverse = new FlinkUniverse(statefulFunctionsUniverse);
+    FlinkUniverse flinkUniverse = new FlinkUniverse(statefulFunctionsUniverse, 
stateFunConfig);
     flinkUniverse.configure(env);
 
-    String jobName = 
configuration.getValue(StatefulFunctionsJobConstants.FLINK_JOB_NAME);
-    env.execute(jobName);
-  }
-
-  private static void setDefaultConfiguration(
-      Configuration configuration, StreamExecutionEnvironment env) {
-    env.getConfig().setGlobalJobParameters(configuration);
-    env.getConfig().enableObjectReuse();
-    final long checkpointingInterval =
-        
configuration.getLong(StatefulFunctionsJobConstants.CHECKPOINTING_INTERVAL);
-    if (checkpointingInterval > 0) {
-      env.enableCheckpointing(checkpointingInterval);
 
 Review comment:
   How would the user set the checkpointing interval now?
   Is there a Flink specific key that users can put in `flink-conf.yaml` ?
   if so, then I would re-emphasize the need to actually pass here the already 
parsed `flink-conf.yaml`

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

Reply via email to