You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2020/06/02 16:14:04 UTC

[nifi] branch master updated: NIFI-7447: When returning an object from a Controller Service, if that object is defined as an interface, proxy that interface. This way, any method call into the object will also change the classloader to the appropriate classloader.

This is an automated email from the ASF dual-hosted git repository.

bbende pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/master by this push:
     new e31c323  NIFI-7447: When returning an object from a Controller Service, if that object is defined as an interface, proxy that interface. This way, any method call into the object will also change the classloader to the appropriate classloader.
e31c323 is described below

commit e31c323aa70dfe8f1bc7a3f39c216f9154529654
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Wed May 13 12:04:44 2020 -0400

    NIFI-7447: When returning an object from a Controller Service, if that object is defined as an interface, proxy that interface. This way, any method call into the object will also change the classloader to the appropriate classloader.
---
 .../controller/ControllerServiceProxyWrapper.java  |  56 ++++++
 ...StandardControllerServiceInvocationHandler.java | 187 ++++++++++++++++++++-
 ...StandardControllerServiceInvocationHandler.java | 161 ++++++++++++++++++
 3 files changed, 397 insertions(+), 7 deletions(-)

diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/ControllerServiceProxyWrapper.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/ControllerServiceProxyWrapper.java
new file mode 100644
index 0000000..3a7bd75
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/ControllerServiceProxyWrapper.java
@@ -0,0 +1,56 @@
+/*
+ * 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.nifi.controller;
+
+/**
+ * An interface that can be added to a Proxy object for a Controller Service in order to get the underlying object that is being proxied.
+ * This is done so that any object that is returned by a Controller Service
+ * can be unwrapped if it's passed back to the Controller Service. This is needed in order to accommodate for the following scenario.
+ * Consider a Controller Service that has two methods:
+ *
+ * <pre><code>
+ * public MyObject createObject();
+ * public void useObject(MyObject object);
+ * </code></pre>
+ *
+ * And further consider that MyObject is an interface with multiple implementations.
+ * If the {@code useObject} method is implemented using logic such as:
+ *
+ * <pre><code>
+ * public void useObject(final MyObject object) {
+ *       if (object instanceof SomeObject) {
+ *           // perform some action
+ *       }
+ * }
+ * </code></pre>
+ *
+ * In this case, if the {@code createObject} method does in fact create an instance of {@code SomeObject}, the proxied object that is returned will not be of type {@code SomeObject}
+ * because {@code SomeObject} is a class, not an interface. So the proxy implements the {@code MyObject} interface, but it is not an instance of {@code SomeObject}.
+ * As a result, the instanceof check would return {@code false} but the service implementor should reasonably expect it to return {@code true}.
+ * In order to accommodate this behavior, this interface can be added to the proxy and then the underlying object can be "unwrapped" when being provided to the Controller Service.
+ *
+ * The {@link java.lang.reflect.InvocationHandler InvocationHandler} is then able to implement the method in order to unwrap the object.
+ *
+ * @param <T> the type of the wrapped object
+ */
+public interface ControllerServiceProxyWrapper<T> {
+    /**
+     * @return the object that is being wrapped/proxied.
+     */
+    T getWrapped();
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java
index 1347e78..ef0cdf1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java
@@ -16,30 +16,42 @@
  */
 package org.apache.nifi.controller.service;
 
+import org.apache.commons.lang3.ClassUtils;
 import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.ControllerServiceProxyWrapper;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
 
 public class StandardControllerServiceInvocationHandler implements ControllerServiceInvocationHandler {
+    private static final Logger logger = LoggerFactory.getLogger(StandardControllerServiceInvocationHandler.class);
+    private static final Method PROXY_WRAPPER_GET_WRAPPED_METHOD;
 
     private static final Set<Method> validDisabledMethods;
     static {
         // methods that are okay to be called when the service is disabled.
         final Set<Method> validMethods = new HashSet<>();
-        for (final Method method : ControllerService.class.getMethods()) {
-            validMethods.add(method);
-        }
-        for (final Method method : Object.class.getMethods()) {
-            validMethods.add(method);
-        }
+        validMethods.addAll(Arrays.asList(ControllerService.class.getMethods()));
+        validMethods.addAll(Arrays.asList(Object.class.getMethods()));
         validDisabledMethods = Collections.unmodifiableSet(validMethods);
+
+        try {
+            PROXY_WRAPPER_GET_WRAPPED_METHOD = ControllerServiceProxyWrapper.class.getMethod("getWrapped");
+        } catch (final NoSuchMethodException e) {
+            throw new AssertionError("Could not find getWrapped Method for ProxyWrapper");
+        }
     }
 
     private final ControllerService originalService;
@@ -72,6 +84,8 @@ public class StandardControllerServiceInvocationHandler implements ControllerSer
     public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
         final String methodName = method.getName();
         if ("initialize".equals(methodName) || "onPropertyModified".equals(methodName)) {
+            // We can blindly throw UnsupportedOperationException because these methods will only ever be invoked by the framework directly
+            // on the controller service implementation, not on this proxy object.
             throw new UnsupportedOperationException(method + " may only be invoked by the NiFi framework");
         }
 
@@ -83,8 +97,22 @@ public class StandardControllerServiceInvocationHandler implements ControllerSer
                 + serviceNodeHolder.get().getIdentifier() + " because the Controller Service's State is currently " + state);
         }
 
+        final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
         try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, originalService.getClass(), originalService.getIdentifier())) {
-            return method.invoke(originalService, args);
+            // If any objects are proxied, unwrap them so that we provide the unproxied object to the Controller Service.
+            final Object[] unwrapped = unwrapProxies(args, Thread.currentThread().getContextClassLoader(), method);
+
+            // Invoke the method on the underlying implementation
+            final Object returnedFromImpl = method.invoke(originalService, unwrapped);
+
+            // If the return object is known to the caller, it can be returned directly. Otherwise, proxy the object so that
+            // calls into the proxy are called through the appropriate ClassLoader.
+            if (returnedFromImpl == null || isInHierarchy(returnedFromImpl.getClass().getClassLoader(), originalClassLoader)) {
+                return returnedFromImpl;
+            }
+
+            // Proxy the return object, if necessary, and return the proxy.
+            return proxy(returnedFromImpl, method.getReturnType());
         } catch (final InvocationTargetException e) {
             // If the ControllerService throws an Exception, it'll be wrapped in an InvocationTargetException. We want
             // to instead re-throw what the ControllerService threw, so we pull it out of the InvocationTargetException.
@@ -92,4 +120,149 @@ public class StandardControllerServiceInvocationHandler implements ControllerSer
         }
     }
 
+    private boolean isInHierarchy(final ClassLoader objectClassLoader, final ClassLoader classLoaderHierarchy) {
+        if (classLoaderHierarchy == null) {
+            return false;
+        }
+        if (objectClassLoader == classLoaderHierarchy) {
+            return true;
+        }
+        return isInHierarchy(objectClassLoader, classLoaderHierarchy.getParent());
+    }
+
+    private Object proxy(final Object toProxy, final Class<?> declaredType) {
+        if (toProxy == null) {
+            return null;
+        }
+
+        // We only want to proxy the object if the object is defined by the method that
+        // was invoked as being an interface. For example, if a method is expected to return a java.lang.String,
+        // we do not want to instead return a proxy because the Proxy won't be a String.
+        if (declaredType == null || !declaredType.isInterface()) {
+            return toProxy;
+        }
+
+        // If the ClassLoader is null, we have a primitive type, which we can't proxy.
+        if (toProxy.getClass().getClassLoader() == null) {
+            return toProxy;
+        }
+
+        // The proxy that is to be returned needs to ensure that it implements all interfaces that are defined by the
+        // object. We cannot simply implement the return that that is defined, because the code that receives the object
+        // may perform further inspection. For example, consider that a javax.jms.Message is returned. If this method proxies
+        // only that method, but the object itself is a javax.jms.BytesMessage, then code such as the following will result in `isBytes == false`
+        // when it should be `true`:
+        //
+        // final javax.jms.Message myMessage = controllerService.getMessage();
+        // final boolean isBytes = myMessage instanceof javax.jms.BytesMessage;
+        final List<Class<?>> interfaces = ClassUtils.getAllInterfaces(toProxy.getClass());
+        if (interfaces == null || interfaces.isEmpty()) {
+            return toProxy;
+        }
+
+        // Add the ControllerServiceProxyWrapper to the List of interfaces to implement. See javadocs for ControllerServiceProxyWrapper
+        // to understand why this is needed.
+        if (!interfaces.contains(ControllerServiceProxyWrapper.class)) {
+            interfaces.add(ControllerServiceProxyWrapper.class);
+        }
+
+        final Class<?>[] interfaceTypes = interfaces.toArray(new Class<?>[0]);
+        final InvocationHandler invocationHandler = new ProxiedReturnObjectInvocationHandler(toProxy);
+        return Proxy.newProxyInstance(toProxy.getClass().getClassLoader(), interfaceTypes, invocationHandler);
+    }
+
+    private Object[] unwrapProxies(final Object[] values, final ClassLoader expectedClassLoader, final Method method) {
+        if (!containsWrappedProxy(values)) {
+            return values;
+        }
+
+        final Object[] unwrappedValues = new Object[values.length];
+        for (int i=0; i < values.length; i++) {
+            unwrappedValues[i] = unwrap(values[i], expectedClassLoader, method);
+        }
+
+        return unwrappedValues;
+    }
+
+    private Object unwrap(final Object value, final ClassLoader expectedClassLoader, final Method method) {
+        if (!isWrappedProxy(value)) {
+            return value;
+        }
+
+        final ControllerServiceProxyWrapper<?> wrapper = (ControllerServiceProxyWrapper<?>) value;
+        final Object wrapped = wrapper.getWrapped();
+        if (wrapped == null) {
+            return null;
+        }
+
+        final ClassLoader wrappedClassLoader = wrapped.getClass().getClassLoader();
+        if (isInHierarchy(wrappedClassLoader, expectedClassLoader)) {
+            logger.trace("Unwrapped {} to be used by {} when calling {}", wrapped, originalService, method);
+            return wrapped;
+        }
+
+        logger.trace("Will not unwrap {} because even though it is a wrapped proxy object, the wrapped object's ClassLoader is {}, not {}", value, wrappedClassLoader, expectedClassLoader);
+        return value;
+    }
+
+    private boolean containsWrappedProxy(final Object[] values) {
+        if (values == null || values.length == 0) {
+            return false;
+        }
+
+        for (final Object value : values) {
+            if (isWrappedProxy(value)) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    private boolean isWrappedProxy(final Object value) {
+        if (value == null) {
+            return false;
+        }
+
+        final Class<?> valueClass = value.getClass();
+        return ControllerServiceProxyWrapper.class.isAssignableFrom(valueClass) && Proxy.isProxyClass(valueClass);
+    }
+
+    private class ProxiedReturnObjectInvocationHandler implements InvocationHandler {
+        private final Object toProxy;
+        private final ClassLoader classLoaderForProxy;
+
+        public ProxiedReturnObjectInvocationHandler(final Object toProxy) {
+            this.toProxy = toProxy;
+            this.classLoaderForProxy = toProxy.getClass().getClassLoader();
+        }
+
+        @Override
+        public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable {
+            if (PROXY_WRAPPER_GET_WRAPPED_METHOD.equals(method)) {
+                return toProxy;
+            }
+
+            final Object[] unwrapped = unwrapProxies(args, classLoaderForProxy, method);
+
+            final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+            try {
+                Thread.currentThread().setContextClassLoader(classLoaderForProxy);
+                final Object returnedFromImpl = method.invoke(toProxy, unwrapped);
+
+                // If the return object is known to the caller, it can be returned directly. Otherwise, proxy the object so that
+                // calls into the proxy are called through the appropriate ClassLoader.
+                if (returnedFromImpl == null || isInHierarchy(returnedFromImpl.getClass().getClassLoader(), originalClassLoader)) {
+                    return returnedFromImpl;
+                }
+
+                return proxy(returnedFromImpl, method.getReturnType());
+            } catch (final InvocationTargetException ite) {
+                throw ite.getCause();
+            } finally {
+                Thread.currentThread().setContextClassLoader(originalClassLoader);
+            }
+        }
+
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceInvocationHandler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceInvocationHandler.java
new file mode 100644
index 0000000..bee0b34
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceInvocationHandler.java
@@ -0,0 +1,161 @@
+/*
+ * 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.nifi.controller.service;
+
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.nar.ExtensionManager;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.lang.reflect.Proxy;
+import java.net.URL;
+import java.net.URLClassLoader;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestStandardControllerServiceInvocationHandler {
+
+    @Before
+    public void setClassLoader() {
+        // Change context class loader to a new, empty class loader so that calls to Controller Service will need to proxy returned objects.
+        final URLClassLoader classLoader = new URLClassLoader(new URL[] {}, null);
+        Thread.currentThread().setContextClassLoader(classLoader);
+    }
+
+    @Test
+    public void testSimpleProxy() {
+        final BaseControllerService proxiedService = createProxyService();
+        assertEquals(1, proxiedService.getLevel());
+
+        final Integer integer = proxiedService.getLevel();
+        assertEquals(Integer.valueOf(1), integer);
+
+        final String toStringValue = proxiedService.toString();
+        assertFalse(Proxy.isProxyClass(toStringValue.getClass()));
+
+        BaseControllerService nextLevel = proxiedService.getNextLevel();
+        assertNotNull(nextLevel);
+        assertEquals(2, nextLevel.getLevel());
+        assertFalse(nextLevel instanceof TestService);
+        assertTrue(Proxy.isProxyClass(nextLevel.getClass()));
+
+        final BaseControllerService thirdLevel = nextLevel.getNextLevel();
+        assertNotNull(thirdLevel);
+        assertEquals(3, thirdLevel.getLevel());
+        assertFalse(thirdLevel instanceof TestService);
+        assertTrue(Proxy.isProxyClass(nextLevel.getClass()));
+
+        for (int i=0; i < 5; i++) {
+            assertEquals(i + 2, nextLevel.getLevel());
+            assertFalse(nextLevel instanceof TestService);
+            assertTrue(Proxy.isProxyClass(nextLevel.getClass()));
+            nextLevel = nextLevel.getNextLevel();
+        }
+    }
+
+
+    @Test
+    public void testObjectsUsedWithinProxyNotProxied() {
+        final BaseControllerService proxiedService = createProxyService();
+        assertEquals(1, proxiedService.getLevel());
+
+        proxiedService.assertNotProxied();
+    }
+
+    @Test
+    public void testObjectsCreatedByServiceNotProxiedWhenHandedBack() {
+        final BaseControllerService proxiedService = createProxyService();
+        assertEquals(1, proxiedService.getLevel());
+
+        final BaseControllerService nextLevel = proxiedService.getNextLevel();
+
+        // The nextLevel that is returned should be proxied. This ensures that any method invoked on this
+        // class is invoked with the appropriate thread context class loader in place.
+        assertTrue(Proxy.isProxyClass(nextLevel.getClass()));
+
+        // When the proxied object is handed back to the Controller Service, though, the service should receive a
+        // version that is not proxied.
+        proxiedService.assertNotProxied(nextLevel);
+    }
+
+
+    private BaseControllerService createProxyService() {
+        final ExtensionManager extensionManager = Mockito.mock(ExtensionManager.class);
+        final TestService testService = new TestService();
+        testService.setLevel(1);
+
+        final ControllerServiceNode serviceNode = Mockito.mock(ControllerServiceNode.class);
+        Mockito.when(serviceNode.getState()).thenReturn(ControllerServiceState.ENABLED);
+
+        final StandardControllerServiceInvocationHandler handler = new StandardControllerServiceInvocationHandler(extensionManager, testService);
+        handler.setServiceNode(serviceNode);
+
+        return (BaseControllerService) Proxy.newProxyInstance(getClass().getClassLoader(), new Class<?>[] {BaseControllerService.class}, handler);
+    }
+
+
+    public interface BaseControllerService extends ControllerService {
+        BaseControllerService getNextLevel();
+        int getLevel();
+        void assertNotProxied();
+        void assertNotProxied(BaseControllerService service);
+    }
+
+    public static class TestService extends AbstractControllerService implements BaseControllerService {
+        private int level;
+
+        @Override
+        public BaseControllerService getNextLevel() {
+            final TestService nextLevel = new TestService();
+            nextLevel.setLevel(level + 1);
+            return nextLevel;
+        }
+
+        @Override
+        public int getLevel() {
+            return level;
+        }
+
+        public void setLevel(final int level) {
+            this.level = level;
+        }
+
+        @Override
+        public void assertNotProxied() {
+            BaseControllerService nextLevel = getNextLevel();
+            for (int i=0; i < 5; i++) {
+                assertEquals(level + i + 1, nextLevel.getLevel());
+                assertTrue(nextLevel instanceof TestService);
+                assertFalse(Proxy.isProxyClass(nextLevel.getClass()));
+
+                nextLevel = nextLevel.getNextLevel();
+            }
+        }
+
+        @Override
+        public void assertNotProxied(final BaseControllerService service) {
+            assertTrue(service instanceof TestService);
+            assertFalse(Proxy.isProxyClass(service.getClass()));
+        }
+    }
+}