Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-24558][API/DataStream]make parent ClassLoader variable which c… #17521

Closed
wants to merge 9 commits into from
Expand Up @@ -30,6 +30,7 @@
import org.apache.flink.configuration.DeploymentOptions;
import org.apache.flink.core.execution.PipelineExecutorServiceLoader;
import org.apache.flink.runtime.client.JobInitializationException;
import org.apache.flink.runtime.execution.librarycache.ClassLoaderFactoryBuilder;
import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders;
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.util.ExceptionUtils;
Expand All @@ -41,8 +42,10 @@

import java.net.URL;
import java.net.URLClassLoader;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.ServiceLoader;

import static org.apache.flink.util.FlinkUserCodeClassLoader.NOOP_EXCEPTION_HANDLER;
import static org.apache.flink.util.Preconditions.checkNotNull;
Expand Down Expand Up @@ -70,6 +73,23 @@ public static URLClassLoader buildUserCodeClassLoader(
FlinkUserCodeClassLoaders.ResolveOrder.fromString(classLoaderResolveOrder);
final boolean checkClassloaderLeak =
configuration.getBoolean(CoreOptions.CHECK_LEAKED_CLASSLOADER);

ServiceLoader<ClassLoaderFactoryBuilder> classLoaderService =
ServiceLoader.load(ClassLoaderFactoryBuilder.class);
Iterator<ClassLoaderFactoryBuilder> factoryIt = classLoaderService.iterator();
ClassLoaderFactoryBuilder factory = null;
while (factoryIt.hasNext()) {
factory = factoryIt.next();
if (factory.isCompatible()) {
return factory.buildClientLoaderFactory(
resolveOrder,
alwaysParentFirstLoaderPatterns,
NOOP_EXCEPTION_HANDLER,
checkClassloaderLeak)
.createClassLoader(urls);
}
}

return FlinkUserCodeClassLoaders.create(
resolveOrder,
urls,
Expand Down
Expand Up @@ -20,18 +20,26 @@

import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.client.TestingClientClassLoaderFactoryBuilder.TestingURLClassLoader;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.core.testutils.CommonTestUtils;
import org.apache.flink.runtime.client.JobInitializationException;
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.util.FlinkUserCodeClassLoader;
import org.apache.flink.util.SerializedThrowable;
import org.apache.flink.util.TestLogger;

import org.junit.Assert;
import org.junit.Test;

import java.net.URLClassLoader;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;

import static org.junit.Assert.assertTrue;

/** Test for the ClientUtils. */
public class ClientUtilsTest extends TestLogger {

Expand Down Expand Up @@ -128,4 +136,47 @@ public void testWaitUntilJobInitializationFinished_regular() throws Exception {
},
ClassLoader.getSystemClassLoader());
}

@Test
public void testClassLoader_by_customized_classLoaderFactory() throws Exception {
System.getProperties()
.setProperty(
TestingClientClassLoaderFactoryBuilder
.TESTING_CLASSLOADER_FACTORY_BUILDER_ENABLE,
Boolean.TRUE.toString());
URLClassLoader classLoader = null;
try {
Configuration configuration = new Configuration();
classLoader =
ClientUtils.buildUserCodeClassLoader(
Collections.emptyList(),
Collections.emptyList(),
this.getClass().getClassLoader(),
configuration);
assertTrue(
"The impl class must be " + TestingURLClassLoader.class.getName(),
classLoader instanceof TestingURLClassLoader);
} finally {
System.clearProperty(
TestingClientClassLoaderFactoryBuilder
.TESTING_CLASSLOADER_FACTORY_BUILDER_ENABLE);
}
}

@Test
public void testClassLoader_without_customized_classLoaderFactory() {
URLClassLoader classLoader = null;

Configuration configuration = new Configuration();
configuration.setBoolean(CoreOptions.CHECK_LEAKED_CLASSLOADER, false);
classLoader =
ClientUtils.buildUserCodeClassLoader(
Collections.emptyList(),
Collections.emptyList(),
this.getClass().getClassLoader(),
configuration);
assertTrue(
"The impl class must be " + FlinkUserCodeClassLoader.class.getName(),
classLoader instanceof FlinkUserCodeClassLoader);
}
}
@@ -0,0 +1,74 @@
/*
* 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.client;

import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager.ClassLoaderFactory;
import org.apache.flink.runtime.execution.librarycache.ClassLoaderFactoryBuilder;
import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders.ResolveOrder;

import javax.annotation.Nullable;

import java.net.URL;
import java.net.URLClassLoader;
import java.util.function.Consumer;

/** Testing implementation of {@link ClassLoaderFactoryBuilder}. */
public class TestingClientClassLoaderFactoryBuilder implements ClassLoaderFactoryBuilder {
public static final String TESTING_CLASSLOADER_FACTORY_BUILDER_ENABLE =
"testing_classloader_factory_builder_enable";

@Override
public boolean isCompatible() {
return Boolean.parseBoolean(
System.getProperties().getProperty(TESTING_CLASSLOADER_FACTORY_BUILDER_ENABLE));
}

@Override
public ClassLoaderFactory buildServerLoaderFactory(
ResolveOrder classLoaderResolveOrder,
String[] alwaysParentFirstPatterns,
@Nullable Consumer<Throwable> exceptionHander,
boolean checkClassLoaderLeak) {
throw new UnsupportedOperationException();
}

@Override
public ClassLoaderFactory buildClientLoaderFactory(
ResolveOrder classLoaderResolveOrder,
String[] alwaysParentFirstPatterns,
@Nullable Consumer<Throwable> exceptionHander,
boolean checkClassLoaderLeak) {
return new TestingClassLoaderFactory();
}

private static class TestingClassLoaderFactory implements ClassLoaderFactory {
@Override
public URLClassLoader createClassLoader(URL[] libraryURLs) {
return new TestingURLClassLoader(
libraryURLs, Thread.currentThread().getContextClassLoader());
}
}

public static class TestingURLClassLoader extends URLClassLoader {

public TestingURLClassLoader(URL[] urls, ClassLoader parent) {
super(urls, parent);
}
}
}
@@ -0,0 +1,16 @@
# 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.

org.apache.flink.client.TestingClientClassLoaderFactoryBuilder
Expand Up @@ -41,7 +41,9 @@
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
import java.util.ServiceLoader;
import java.util.Set;
import java.util.function.Consumer;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -125,30 +127,31 @@ public void shutdown() {
}

// --------------------------------------------------------------------------------------------

/** Customize classLoader factory. */
@FunctionalInterface
public interface ClassLoaderFactory {
URLClassLoader createClassLoader(URL[] libraryURLs);
}

private static final class DefaultClassLoaderFactory implements ClassLoaderFactory {
/** Default ClassLoader factory. */
public static class DefaultClassLoaderFactory implements ClassLoaderFactory {

/** The resolve order to use when creating a {@link ClassLoader}. */
private final FlinkUserCodeClassLoaders.ResolveOrder classLoaderResolveOrder;
protected final FlinkUserCodeClassLoaders.ResolveOrder classLoaderResolveOrder;

/**
* List of patterns for classes that should always be resolved from the parent ClassLoader,
* if possible.
*/
private final String[] alwaysParentFirstPatterns;
protected final String[] alwaysParentFirstPatterns;

/** Class loading exception handler. */
private final Consumer<Throwable> classLoadingExceptionHandler;
protected final Consumer<Throwable> classLoadingExceptionHandler;

/** Test if classloader is used outside of job. */
private final boolean checkClassLoaderLeak;
protected final boolean checkClassLoaderLeak;

private DefaultClassLoaderFactory(
protected DefaultClassLoaderFactory(
FlinkUserCodeClassLoaders.ResolveOrder classLoaderResolveOrder,
String[] alwaysParentFirstPatterns,
Consumer<Throwable> classLoadingExceptionHandler,
Expand All @@ -164,22 +167,43 @@ public URLClassLoader createClassLoader(URL[] libraryURLs) {
return FlinkUserCodeClassLoaders.create(
classLoaderResolveOrder,
libraryURLs,
FlinkUserCodeClassLoaders.class.getClassLoader(),
getParentClassLoader(),
alwaysParentFirstPatterns,
classLoadingExceptionHandler,
checkClassLoaderLeak);
}

protected ClassLoader getParentClassLoader() {
return FlinkUserCodeClassLoaders.class.getClassLoader();
}
}

public static ClassLoaderFactory defaultClassLoaderFactory(
FlinkUserCodeClassLoaders.ResolveOrder classLoaderResolveOrder,
String[] alwaysParentFirstPatterns,
@Nullable FatalErrorHandler fatalErrorHandlerJvmMetaspaceOomError,
boolean checkClassLoaderLeak) {
Consumer<Throwable> exceptionHandler =
createClassLoadingExceptionHandler(fatalErrorHandlerJvmMetaspaceOomError);
ServiceLoader<ClassLoaderFactoryBuilder> classLoaderService =
ServiceLoader.load(ClassLoaderFactoryBuilder.class);
Iterator<ClassLoaderFactoryBuilder> factoryIt = classLoaderService.iterator();
ClassLoaderFactoryBuilder factory = null;
while (factoryIt.hasNext()) {
factory = factoryIt.next();
if (factory.isCompatible()) {
return factory.buildServerLoaderFactory(
classLoaderResolveOrder,
alwaysParentFirstPatterns,
exceptionHandler,
checkClassLoaderLeak);
}
}

return new DefaultClassLoaderFactory(
classLoaderResolveOrder,
alwaysParentFirstPatterns,
createClassLoadingExceptionHandler(fatalErrorHandlerJvmMetaspaceOomError),
exceptionHandler,
checkClassLoaderLeak);
}

Expand Down
@@ -0,0 +1,54 @@
/*
* 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.runtime.execution.librarycache;

import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager.ClassLoaderFactory;

import javax.annotation.Nullable;

import java.util.function.Consumer;

/**
* Provides facilities to customize {@link ClassLoaderFactory} by user ,it is instantinate by
* ServiceLoader.
*
* @see java.util.ServiceLoader
*/
public interface ClassLoaderFactoryBuilder {

public default boolean isCompatible() {
return false;
}

public default ClassLoaderFactory buildServerLoaderFactory(
FlinkUserCodeClassLoaders.ResolveOrder classLoaderResolveOrder,
String[] alwaysParentFirstPatterns,
@Nullable Consumer<Throwable> exceptionHander,
boolean checkClassLoaderLeak) {
throw new UnsupportedOperationException();
}

public default ClassLoaderFactory buildClientLoaderFactory(
FlinkUserCodeClassLoaders.ResolveOrder classLoaderResolveOrder,
String[] alwaysParentFirstPatterns,
@Nullable Consumer<Throwable> exceptionHander,
boolean checkClassLoaderLeak) {
throw new UnsupportedOperationException();
}
}
Expand Up @@ -84,7 +84,7 @@ public static URLClassLoader create(
}
}

private static URLClassLoader wrapWithSafetyNet(
public static URLClassLoader wrapWithSafetyNet(
FlinkUserCodeClassLoader classLoader, boolean check) {
return check
? new SafetyNetWrapperClassLoader(classLoader, classLoader.getParent())
Expand Down