From a68f71e9fca0d5a22dc146c2a7bb3a0965212659 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Fri, 10 Jan 2025 22:20:34 +0800 Subject: [PATCH] [core] Do not use Collections.unmodifiableMap to avoid stack overflow --- .../paimon/metrics/MetricGroupImpl.java | 5 ++- .../org/apache/paimon/schema/TableSchema.java | 2 +- .../apache/paimon/schema/TableSchemaTest.java | 36 ++++++++++++++----- .../flink/metrics/FlinkMetricGroup.java | 3 +- .../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 fb4a02d04ef6..d8d05d2ae58e 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 @@ 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 String getGroupName() { @Override public Map getAllVariables() { - return Collections.unmodifiableMap(variables); + return variables; } @Override @@ -117,7 +116,7 @@ private Metric addMetric(String metricName, Metric metric) { @Override public Map 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 a38340a6e510..e7a0dbd9f1e2 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 TableSchema( 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 d7f2e660a05a..25d4601c6b01 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.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 @@ /** Test for {@link TableSchema}. */ public class TableSchemaTest { + @Test + public void testTableSchemaCopy() { + Map 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 fields = @@ -50,8 +69,8 @@ public void testCrossPartition() { new DataField(0, "f0", DataTypes.INT()), new DataField(1, "f1", DataTypes.INT()), new DataField(2, "f2", DataTypes.INT())); - List partitionKeys = Collections.singletonList("f0"); - List primaryKeys = Collections.singletonList("f1"); + List partitionKeys = singletonList("f0"); + List primaryKeys = singletonList("f1"); Map options = new HashMap<>(); TableSchema schema = @@ -130,8 +149,8 @@ public void testSequenceField() { new DataField(1, "f1", DataTypes.INT()), new DataField(2, "f2", DataTypes.INT()), new DataField(3, "f3", DataTypes.INT())); - List partitionKeys = Collections.singletonList("f0"); - List primaryKeys = Collections.singletonList("f1"); + List partitionKeys = singletonList("f0"); + List primaryKeys = singletonList("f1"); Map options = new HashMap<>(); TableSchema schema = @@ -168,7 +187,7 @@ public void testFieldsPrefix() { new DataField(0, "f0", DataTypes.INT()), new DataField(1, "f1", DataTypes.INT()), new DataField(2, "f2", DataTypes.INT())); - List primaryKeys = Collections.singletonList("f0"); + List primaryKeys = singletonList("f0"); Map 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 void testBucket() { new DataField(0, "f0", DataTypes.INT()), new DataField(1, "f1", DataTypes.INT()), new DataField(2, "f2", DataTypes.INT())); - List partitionKeys = Collections.singletonList("f0"); - List primaryKeys = Collections.singletonList("f1"); + List partitionKeys = singletonList("f0"); + List primaryKeys = singletonList("f1"); Map options = new HashMap<>(); TableSchema schema = @@ -200,7 +219,6 @@ public void testBucket() { 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 9cf1840230eb..60c27251a30f 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.Metric; import org.apache.paimon.metrics.MetricGroup; -import java.util.Collections; import java.util.Map; /** @@ -75,7 +74,7 @@ public Histogram histogram(String name, int windowSize) { @Override public Map 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 64bdc8140b71..ec9420dca4ff 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.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 @@ protected void dropDatabaseImpl(String name) { protected void alterDatabaseImpl(String name, List changes) { try { Database database = clients.run(client -> client.getDatabase(name)); - Map parameter = Maps.newHashMap(); - parameter.putAll(database.getParameters()); + Map parameter = new HashMap<>(database.getParameters()); Pair, Set> setPropertiesToRemoveKeys = PropertyChange.getSetPropertiesToRemoveKeys(changes); Map setProperties = setPropertiesToRemoveKeys.getLeft(); Set 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 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);