matriv commented on a change in pull request #18980:
URL: https://github.com/apache/flink/pull/18980#discussion_r825851822



##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java
##########
@@ -60,44 +57,21 @@
     private static final EnvironmentSettings DEFAULT_BATCH_MODE_SETTINGS =
             EnvironmentSettings.newInstance().inBatchMode().build();
 
-    public static final String DEFAULT_BUILTIN_CATALOG = "default_catalog";
-    public static final String DEFAULT_BUILTIN_DATABASE = "default_database";
-
-    /** Factory identifier of the {@link Planner} to use. */
-    private final String planner;
-
-    /** Factory identifier of the {@link Executor} to use. */
-    private final String executor;
-
-    /**
-     * Specifies the name of the initial catalog to be created when 
instantiating {@link
-     * TableEnvironment}.
-     */
-    private final String builtInCatalogName;
-
     /**
-     * Specifies the name of the default database in the initial catalog to be 
created when
-     * instantiating {@link TableEnvironment}.
+     * Holds all the configuration generated by the builder, together with any 
required additional
+     * configuration.
      */
-    private final String builtInDatabaseName;
-
-    /**
-     * Determines if the table environment should work in a batch ({@code 
false}) or streaming
-     * ({@code true}) mode.
-     */
-    private final boolean isStreamingMode;
-
-    private EnvironmentSettings(
-            String planner,
-            @Nullable String executor,
-            String builtInCatalogName,
-            String builtInDatabaseName,
-            boolean isStreamingMode) {
-        this.planner = planner;
-        this.executor = executor;
-        this.builtInCatalogName = builtInCatalogName;
-        this.builtInDatabaseName = builtInDatabaseName;
-        this.isStreamingMode = isStreamingMode;
+    private final Configuration configuration;
+
+    private EnvironmentSettings(Configuration configuration) {
+        this.configuration = configuration;
+        if (configuration.get(RUNTIME_MODE) == AUTOMATIC) {

Review comment:
       ```
   Exception in thread "main" org.apache.flink.table.api.TableException: 
Unknown runtime mode 'AUTOMATIC'. This is a bug. Please consider filing an 
issue.
        at 
org.apache.flink.table.planner.delegation.DefaultPlannerFactory.create(DefaultPlannerFactory.java:72)
        at 
org.apache.flink.table.factories.PlannerFactoryUtil.createPlanner(PlannerFactoryUtil.java:52)
        at 
org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl.create(StreamTableEnvironmentImpl.java:124)
        at 
org.apache.flink.table.api.bridge.java.StreamTableEnvironment.create(StreamTableEnvironment.java:122)
        at 
org.apache.flink.table.api.bridge.java.StreamTableEnvironment.create(StreamTableEnvironment.java:94)
        at 
org.apache.flink.table.examples.java.basics.StreamSQLExample.main(StreamSQLExample.java:55)
   ```




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

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to