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/flink.git
The following commit(s) were added to refs/heads/master by this push: new ea7a356 [FLINK-25071][parquet] SerializableConfiguration should not load resources ea7a356 is described below commit ea7a356b861dd464d37b57ca10f5aff797afb905 Author: Jingsong Lee <jingsongl...@gmail.com> AuthorDate: Thu Dec 2 17:56:51 2021 +0800 [FLINK-25071][parquet] SerializableConfiguration should not load resources This closes #17987 --- .../parquet/utils/SerializableConfiguration.java | 7 ++-- .../formats/parquet/ParquetFileSystemITCase.java | 2 - .../utils/SerializableConfigurationTest.java | 49 ++++++++++++++++++++++ 3 files changed, 52 insertions(+), 6 deletions(-) diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/SerializableConfiguration.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/SerializableConfiguration.java index 8ba986b..3b4c4ba 100644 --- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/SerializableConfiguration.java +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/SerializableConfiguration.java @@ -30,7 +30,7 @@ public class SerializableConfiguration implements Serializable { private static final long serialVersionUID = 1L; - private Configuration conf; + private transient Configuration conf; public SerializableConfiguration(Configuration conf) { this.conf = conf; @@ -41,13 +41,12 @@ public class SerializableConfiguration implements Serializable { } private void writeObject(ObjectOutputStream out) throws IOException { + out.defaultWriteObject(); conf.write(out); } private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - if (conf == null) { - conf = new Configuration(); - } + conf = new Configuration(false); conf.readFields(in); } } diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetFileSystemITCase.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetFileSystemITCase.java index 2fbdf9c..38475a1 100644 --- a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetFileSystemITCase.java +++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetFileSystemITCase.java @@ -27,7 +27,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -116,7 +115,6 @@ public class ParquetFileSystemITCase extends BatchFileSystemITCaseBase { } } - @Ignore @Test public void testLimitableBulkFormat() throws ExecutionException, InterruptedException { super.tableEnv() diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/utils/SerializableConfigurationTest.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/utils/SerializableConfigurationTest.java new file mode 100644 index 0000000..7f29899 --- /dev/null +++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/utils/SerializableConfigurationTest.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.parquet.utils; + +import org.apache.hadoop.conf.Configuration; +import org.jetbrains.annotations.Nullable; +import org.junit.Test; + +import java.io.IOException; +import java.net.URL; + +import static org.apache.flink.util.InstantiationUtil.deserializeObject; +import static org.apache.flink.util.InstantiationUtil.serializeObject; + +/** Test for {@link SerializableConfiguration}. */ +public class SerializableConfigurationTest { + + @Test + public void testResource() throws IOException, ClassNotFoundException { + ClassLoader cl = + new ClassLoader(Thread.currentThread().getContextClassLoader()) { + @Nullable + @Override + public URL getResource(String name) { + throw new RuntimeException(); + } + }; + SerializableConfiguration conf = + deserializeObject( + serializeObject(new SerializableConfiguration(new Configuration())), cl); + conf.conf().getInt("one-key", 0); + } +}