Skip to content

Commit

Permalink
[FLINK-16917][orc] Workaround for classloader leak in OrcFile.
Browse files Browse the repository at this point in the history
See https://issues.apache.org/jira/browse/ORC-653
OrcFile#getStaticMemoryManager caches initial configuration and leaks classloader in it. Thus, new Flink jobs use the classloader of the first job implicitly.

By adding ThreadLocalClassLoaderConfiguration, which forces the use of thread-local classloader over the initial classloader, Flink jobs use the appropriate classloader on higher runtime costs (no caches).
This commit should be reverted, once the bug in ORC is fixed.
  • Loading branch information
Arvid Heise authored and zentol committed Aug 16, 2020
1 parent e8cfc75 commit 6c130da
Show file tree
Hide file tree
Showing 3 changed files with 65 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.flink.connectors.hive.write.HiveWriterFactory;
import org.apache.flink.formats.parquet.row.ParquetRowDataBuilder;
import org.apache.flink.orc.OrcSplitReaderUtil;
import org.apache.flink.orc.writer.ThreadLocalClassLoaderConfiguration;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
import org.apache.flink.streaming.api.functions.sink.filesystem.HadoopPathBasedBulkFormatBuilder;
Expand Down Expand Up @@ -255,12 +256,14 @@ private Optional<BulkWriter.Factory<RowData>> createBulkWriterFactory(String[] p
formatTypes[i] = tableSchema.getFieldDataType(i).get().getLogicalType();
}
RowType formatType = RowType.of(formatTypes, formatNames);
Configuration formatConf = new Configuration(jobConf);
sd.getSerdeInfo().getParameters().forEach(formatConf::set);
if (serLib.contains("parquet")) {
Configuration formatConf = new Configuration(jobConf);
sd.getSerdeInfo().getParameters().forEach(formatConf::set);
return Optional.of(ParquetRowDataBuilder.createWriterFactory(
formatType, formatConf, hiveVersion.startsWith("3.")));
} else if (serLib.contains("orc")) {
Configuration formatConf = new ThreadLocalClassLoaderConfiguration(jobConf);
sd.getSerdeInfo().getParameters().forEach(formatConf::set);
TypeDescription typeDescription = OrcSplitReaderUtil.logicalTypeToOrcType(formatType);
return Optional.of(hiveShim.createOrcBulkWriterFactory(
formatConf, typeDescription.toString(), formatTypes));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ public BulkWriter<T> create(FSDataOutputStream out) throws IOException {

private OrcFile.WriterOptions getWriterOptions() {
if (null == writerOptions) {
Configuration conf = new Configuration();
Configuration conf = new ThreadLocalClassLoaderConfiguration();
for (Map.Entry<String, String> entry : confMap.entrySet()) {
conf.set(entry.getKey(), entry.getValue());
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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
*
* https://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.orc.writer;

import org.apache.flink.annotation.Internal;

import org.apache.hadoop.conf.Configuration;

import java.net.URL;

/**
* Workaround for https://issues.apache.org/jira/browse/ORC-653.
*
* <p>Since the conf is effectively cached across Flink jobs, at least force the thread local classloader to avoid
* classloader leaks.
*/
@Internal
public final class ThreadLocalClassLoaderConfiguration extends Configuration {
public ThreadLocalClassLoaderConfiguration() {
}

public ThreadLocalClassLoaderConfiguration(Configuration other) {
super(other);
}

@Override
public ClassLoader getClassLoader() {
return Thread.currentThread().getContextClassLoader();
}

@Override
public Class<?> getClassByNameOrNull(String name) {
try {
return Class.forName(name, true, getClassLoader());
} catch (ClassNotFoundException e) {
return null;
}
}

@Override
public URL getResource(String name) {
return getClassLoader().getResource(name);
}
}

0 comments on commit 6c130da

Please sign in to comment.