Skip to content

Commit

Permalink
[core] Do not use Collections.unmodifiableMap to avoid stack overflow (
Browse files Browse the repository at this point in the history
  • Loading branch information
JingsongLi authored Jan 12, 2025
1 parent 6c54255 commit 4696879
Show file tree
Hide file tree
Showing 5 changed files with 35 additions and 22 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.Collections;
import java.util.HashMap;
import java.util.Map;

Expand Down Expand Up @@ -51,7 +50,7 @@ public String getGroupName() {

@Override
public Map<String, String> getAllVariables() {
return Collections.unmodifiableMap(variables);
return variables;
}

@Override
Expand Down Expand Up @@ -117,7 +116,7 @@ private Metric addMetric(String metricName, Metric metric) {

@Override
public Map<String, Metric> getMetrics() {
return Collections.unmodifiableMap(metrics);
return metrics;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -43,15 +45,32 @@
/** 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 =
Arrays.asList(
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 =
Expand Down Expand Up @@ -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<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 =
Expand Down Expand Up @@ -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<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");
Expand All @@ -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<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 =
Expand All @@ -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())));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import org.apache.paimon.metrics.Metric;
import org.apache.paimon.metrics.MetricGroup;

import java.util.Collections;
import java.util.Map;

/**
Expand Down Expand Up @@ -75,7 +74,7 @@ public Histogram histogram(String name, int windowSize) {

@Override
public Map<String, String> getAllVariables() {
return Collections.unmodifiableMap(variables);
return variables;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -588,20 +587,18 @@ protected void dropDatabaseImpl(String name) {
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);
Expand Down

0 comments on commit 4696879

Please sign in to comment.