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

lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/paimon.git


The following commit(s) were added to refs/heads/master by this push:
     new 4696879375 [core] Do not use Collections.unmodifiableMap to avoid 
stack overflow (#4883)
4696879375 is described below

commit 46968793755ff037cb742f307a66582fd38a7d70
Author: Jingsong Lee <[email protected]>
AuthorDate: Sun Jan 12 11:51:46 2025 +0800

    [core] Do not use Collections.unmodifiableMap to avoid stack overflow 
(#4883)
---
 .../org/apache/paimon/metrics/MetricGroupImpl.java |  5 ++-
 .../java/org/apache/paimon/schema/TableSchema.java |  2 +-
 .../org/apache/paimon/schema/TableSchemaTest.java  | 36 ++++++++++++++++------
 .../paimon/flink/metrics/FlinkMetricGroup.java     |  3 +-
 .../java/org/apache/paimon/hive/HiveCatalog.java   | 11 +++----
 5 files changed, 35 insertions(+), 22 deletions(-)

diff --git 
a/paimon-core/src/main/java/org/apache/paimon/metrics/MetricGroupImpl.java 
b/paimon-core/src/main/java/org/apache/paimon/metrics/MetricGroupImpl.java
index fb4a02d04e..d8d05d2ae5 100644
--- a/paimon-core/src/main/java/org/apache/paimon/metrics/MetricGroupImpl.java
+++ b/paimon-core/src/main/java/org/apache/paimon/metrics/MetricGroupImpl.java
@@ -21,7 +21,6 @@ package org.apache.paimon.metrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -51,7 +50,7 @@ public class MetricGroupImpl implements MetricGroup {
 
     @Override
     public Map<String, String> getAllVariables() {
-        return Collections.unmodifiableMap(variables);
+        return variables;
     }
 
     @Override
@@ -117,7 +116,7 @@ public class MetricGroupImpl implements MetricGroup {
 
     @Override
     public Map<String, Metric> getMetrics() {
-        return Collections.unmodifiableMap(metrics);
+        return metrics;
     }
 
     @Override
diff --git 
a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java 
b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java
index a38340a6e5..e7a0dbd9f1 100644
--- a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java
+++ b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java
@@ -115,7 +115,7 @@ public class TableSchema implements Serializable {
         this.highestFieldId = highestFieldId;
         this.partitionKeys = partitionKeys;
         this.primaryKeys = primaryKeys;
-        this.options = Collections.unmodifiableMap(options);
+        this.options = options;
         this.comment = comment;
         this.timeMillis = timeMillis;
 
diff --git 
a/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java 
b/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java
index d7f2e660a0..25d4601c6b 100644
--- a/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java
+++ b/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java
@@ -31,6 +31,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static java.util.Collections.emptyList;
+import static java.util.Collections.singletonList;
 import static org.apache.paimon.CoreOptions.AGG_FUNCTION;
 import static org.apache.paimon.CoreOptions.BUCKET;
 import static org.apache.paimon.CoreOptions.FIELDS_PREFIX;
@@ -43,6 +45,23 @@ import static 
org.assertj.core.api.Assertions.assertThatThrownBy;
 /** Test for {@link TableSchema}. */
 public class TableSchemaTest {
 
+    @Test
+    public void testTableSchemaCopy() {
+        Map<String, String> options = new HashMap<>();
+        options.put("my-key", "my-value");
+        TableSchema schema =
+                new TableSchema(
+                        1,
+                        singletonList(new DataField(0, "f0", DataTypes.INT())),
+                        10,
+                        emptyList(),
+                        emptyList(),
+                        options,
+                        "");
+        schema = schema.copy(schema.options());
+        assertThat(schema.options()).isSameAs(options);
+    }
+
     @Test
     public void testCrossPartition() {
         List<DataField> fields =
@@ -50,8 +69,8 @@ public class TableSchemaTest {
                         new DataField(0, "f0", DataTypes.INT()),
                         new DataField(1, "f1", DataTypes.INT()),
                         new DataField(2, "f2", DataTypes.INT()));
-        List<String> partitionKeys = Collections.singletonList("f0");
-        List<String> primaryKeys = Collections.singletonList("f1");
+        List<String> partitionKeys = singletonList("f0");
+        List<String> primaryKeys = singletonList("f1");
         Map<String, String> options = new HashMap<>();
 
         TableSchema schema =
@@ -130,8 +149,8 @@ public class TableSchemaTest {
                         new DataField(1, "f1", DataTypes.INT()),
                         new DataField(2, "f2", DataTypes.INT()),
                         new DataField(3, "f3", DataTypes.INT()));
-        List<String> partitionKeys = Collections.singletonList("f0");
-        List<String> primaryKeys = Collections.singletonList("f1");
+        List<String> partitionKeys = singletonList("f0");
+        List<String> primaryKeys = singletonList("f1");
         Map<String, String> options = new HashMap<>();
 
         TableSchema schema =
@@ -168,7 +187,7 @@ public class TableSchemaTest {
                         new DataField(0, "f0", DataTypes.INT()),
                         new DataField(1, "f1", DataTypes.INT()),
                         new DataField(2, "f2", DataTypes.INT()));
-        List<String> primaryKeys = Collections.singletonList("f0");
+        List<String> primaryKeys = singletonList("f0");
         Map<String, String> options = new HashMap<>();
         options.put(MERGE_ENGINE.key(), 
CoreOptions.MergeEngine.AGGREGATE.toString());
         options.put(FIELDS_PREFIX + ".f1." + AGG_FUNCTION, "max");
@@ -186,8 +205,8 @@ public class TableSchemaTest {
                         new DataField(0, "f0", DataTypes.INT()),
                         new DataField(1, "f1", DataTypes.INT()),
                         new DataField(2, "f2", DataTypes.INT()));
-        List<String> partitionKeys = Collections.singletonList("f0");
-        List<String> primaryKeys = Collections.singletonList("f1");
+        List<String> partitionKeys = singletonList("f0");
+        List<String> primaryKeys = singletonList("f1");
         Map<String, String> options = new HashMap<>();
 
         TableSchema schema =
@@ -200,7 +219,6 @@ public class TableSchemaTest {
 
     static RowType newRowType(boolean isNullable, int fieldId) {
         return new RowType(
-                isNullable,
-                Collections.singletonList(new DataField(fieldId, 
"nestedField", DataTypes.INT())));
+                isNullable, singletonList(new DataField(fieldId, 
"nestedField", DataTypes.INT())));
     }
 }
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/metrics/FlinkMetricGroup.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/metrics/FlinkMetricGroup.java
index 9cf1840230..60c27251a3 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/metrics/FlinkMetricGroup.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/metrics/FlinkMetricGroup.java
@@ -24,7 +24,6 @@ import org.apache.paimon.metrics.Histogram;
 import org.apache.paimon.metrics.Metric;
 import org.apache.paimon.metrics.MetricGroup;
 
-import java.util.Collections;
 import java.util.Map;
 
 /**
@@ -75,7 +74,7 @@ public class FlinkMetricGroup implements MetricGroup {
 
     @Override
     public Map<String, String> getAllVariables() {
-        return Collections.unmodifiableMap(variables);
+        return variables;
     }
 
     @Override
diff --git 
a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java
 
b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java
index 64bdc8140b..ec9420dca4 100644
--- 
a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java
+++ 
b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java
@@ -56,7 +56,6 @@ import org.apache.paimon.view.View;
 import org.apache.paimon.view.ViewImpl;
 
 import org.apache.paimon.shade.guava30.com.google.common.collect.Lists;
-import org.apache.paimon.shade.guava30.com.google.common.collect.Maps;
 
 import org.apache.flink.table.hive.LegacyHiveClasses;
 import org.apache.hadoop.conf.Configuration;
@@ -588,20 +587,18 @@ public class HiveCatalog extends AbstractCatalog {
     protected void alterDatabaseImpl(String name, List<PropertyChange> 
changes) {
         try {
             Database database = clients.run(client -> 
client.getDatabase(name));
-            Map<String, String> parameter = Maps.newHashMap();
-            parameter.putAll(database.getParameters());
+            Map<String, String> parameter = new 
HashMap<>(database.getParameters());
             Pair<Map<String, String>, Set<String>> setPropertiesToRemoveKeys =
                     PropertyChange.getSetPropertiesToRemoveKeys(changes);
             Map<String, String> setProperties = 
setPropertiesToRemoveKeys.getLeft();
             Set<String> removeKeys = setPropertiesToRemoveKeys.getRight();
-            if (setProperties.size() > 0) {
+            if (!setProperties.isEmpty()) {
                 parameter.putAll(setProperties);
             }
-            if (removeKeys.size() > 0) {
+            if (!removeKeys.isEmpty()) {
                 parameter.keySet().removeAll(removeKeys);
             }
-            Map<String, String> newProperties = 
Collections.unmodifiableMap(parameter);
-            Database alterDatabase = convertToHiveDatabase(name, 
newProperties);
+            Database alterDatabase = convertToHiveDatabase(name, parameter);
             clients.execute(client -> client.alterDatabase(name, 
alterDatabase));
         } catch (TException e) {
             throw new RuntimeException("Failed to alter database " + name, e);

Reply via email to