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

fanjia pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/incubator-seatunnel.git


The following commit(s) were added to refs/heads/dev by this push:
     new 5a2437c4d [improve] Use ReadonlyConfig to avoid option being changed 
(#4056)
5a2437c4d is described below

commit 5a2437c4d3ed996a528d50639c2439d70db8dabd
Author: Zongwen Li <[email protected]>
AuthorDate: Tue Feb 7 10:00:36 2023 +0800

    [improve] Use ReadonlyConfig to avoid option being changed (#4056)
---
 .../apache/seatunnel/api/table/factory/CatalogFactory.java  |  5 ++---
 .../org/apache/seatunnel/api/table/factory/FactoryUtil.java |  6 +++---
 .../seatunnel/api/table/factory/TableFactoryContext.java    | 13 ++++++-------
 .../org/apache/seatunnel/format/json/JsonFormatFactory.java |  2 +-
 4 files changed, 12 insertions(+), 14 deletions(-)

diff --git 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/CatalogFactory.java
 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/CatalogFactory.java
index 2448c77e2..a813fd665 100644
--- 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/CatalogFactory.java
+++ 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/CatalogFactory.java
@@ -17,14 +17,13 @@
 
 package org.apache.seatunnel.api.table.factory;
 
+import org.apache.seatunnel.api.configuration.ReadonlyConfig;
 import org.apache.seatunnel.api.table.catalog.Catalog;
 
-import java.util.Map;
-
 public interface CatalogFactory extends Factory {
 
     /**
      * Creates a {@link Catalog} using the options.
      */
-    Catalog createCatalog(String catalogName, Map<String, String> options);
+    Catalog createCatalog(String catalogName, ReadonlyConfig options);
 }
diff --git 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java
 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java
index 0f9d1cdff..9874fda13 100644
--- 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java
+++ 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java
@@ -19,6 +19,7 @@ package org.apache.seatunnel.api.table.factory;
 
 import org.apache.seatunnel.api.configuration.Option;
 import org.apache.seatunnel.api.configuration.Options;
+import org.apache.seatunnel.api.configuration.ReadonlyConfig;
 import org.apache.seatunnel.api.configuration.util.OptionRule;
 import org.apache.seatunnel.api.sink.DataSaveMode;
 import org.apache.seatunnel.api.sink.SeaTunnelSink;
@@ -41,7 +42,6 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
 import java.util.ServiceConfigurationError;
 import java.util.ServiceLoader;
 import java.util.stream.Collectors;
@@ -55,7 +55,7 @@ public final class FactoryUtil {
 
     public static <T, SplitT extends SourceSplit, StateT extends Serializable> 
List<SeaTunnelSource<T, SplitT, StateT>> createAndPrepareSource(
         List<CatalogTable> multipleTables,
-        Map<String, String> options,
+        ReadonlyConfig options,
         ClassLoader classLoader,
         String factoryIdentifier) {
 
@@ -90,7 +90,7 @@ public final class FactoryUtil {
     }
 
     public static Catalog createCatalog(String catalogName,
-                                        Map<String, String> options,
+                                        ReadonlyConfig options,
                                         ClassLoader classLoader,
                                         String factoryIdentifier) {
         CatalogFactory catalogFactory = discoverFactory(classLoader, 
CatalogFactory.class, factoryIdentifier);
diff --git 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableFactoryContext.java
 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableFactoryContext.java
index 0c01436ce..97db8a0e6 100644
--- 
a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableFactoryContext.java
+++ 
b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableFactoryContext.java
@@ -17,20 +17,23 @@
 
 package org.apache.seatunnel.api.table.factory;
 
+import org.apache.seatunnel.api.configuration.ReadonlyConfig;
 import org.apache.seatunnel.api.table.catalog.CatalogTable;
 
+import lombok.Getter;
+
 import java.util.List;
-import java.util.Map;
 
 public class TableFactoryContext {
 
     private final List<CatalogTable> catalogTables;
-    private final Map<String, String> options;
+    @Getter
+    private final ReadonlyConfig options;
     private final ClassLoader classLoader;
 
     public TableFactoryContext(
             List<CatalogTable> catalogTables,
-            Map<String, String> options,
+            ReadonlyConfig options,
             ClassLoader classLoader) {
         this.catalogTables = catalogTables;
         this.options = options;
@@ -51,8 +54,4 @@ public class TableFactoryContext {
     public List<CatalogTable> getCatalogTables() {
         return catalogTables;
     }
-
-    public Map<String, String> getOptions() {
-        return this.options;
-    }
 }
diff --git 
a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonFormatFactory.java
 
b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonFormatFactory.java
index 3b1dce8f6..7721897dc 100644
--- 
a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonFormatFactory.java
+++ 
b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonFormatFactory.java
@@ -46,7 +46,7 @@ public class JsonFormatFactory implements 
DeserializationFormatFactory, Serializ
 
     @Override
     public DeserializationFormat 
createDeserializationFormat(TableFactoryContext context) {
-        Map<String, String> options = context.getOptions();
+        Map<String, String> options = context.getOptions().toMap();
         boolean failOnMissingField = 
JsonFormatOptions.getFailOnMissingField(options);
         boolean ignoreParseErrors = 
JsonFormatOptions.getIgnoreParseErrors(options);
 

Reply via email to