You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by js...@apache.org on 2017/08/17 22:59:48 UTC

[1/2] geode git commit: Add test to expose GEODE-3429

Repository: geode
Updated Branches:
  refs/heads/develop 82fad6453 -> 1a67d4627


Add test to expose GEODE-3429


Project: http://git-wip-us.apache.org/repos/asf/geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/geode/commit/1a67d462
Tree: http://git-wip-us.apache.org/repos/asf/geode/tree/1a67d462
Diff: http://git-wip-us.apache.org/repos/asf/geode/diff/1a67d462

Branch: refs/heads/develop
Commit: 1a67d46278ea519a1bfe185a7da11247e9771a4b
Parents: 64f33c3
Author: Jared Stewart <js...@pivotal.io>
Authored: Thu Aug 10 11:21:59 2017 -0700
Committer: Jared Stewart <js...@pivotal.io>
Committed: Thu Aug 17 15:57:59 2017 -0700

----------------------------------------------------------------------
 .../deployment/FunctionScannerTest.java         | 17 ++++++++++
 .../internal/deployment/AbstractFunction.java   | 33 --------------------
 .../internal/deployment/AnnotatedFunction.java  | 23 ++++++++++++++
 .../apache/geode/test/compiler/JarBuilder.java  | 10 ++++--
 .../geode/test/compiler/JavaCompiler.java       | 11 +++++--
 5 files changed, 57 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/1a67d462/geode-core/src/test/java/org/apache/geode/management/internal/deployment/FunctionScannerTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/deployment/FunctionScannerTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/deployment/FunctionScannerTest.java
index af9ffdf..d46b801 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/deployment/FunctionScannerTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/deployment/FunctionScannerTest.java
@@ -23,6 +23,7 @@ import java.net.URL;
 import java.util.Collection;
 
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -95,6 +96,22 @@ public class FunctionScannerTest {
         "org.apache.geode.management.internal.deployment.AbstractImplementsFunction");
   }
 
+  @Test
+  @Ignore("Fails due to GEODE-3429")
+  public void registerFunctionHierarchySplitAcrossTwoJars() throws Exception {
+    File sourceFileOne = loadTestResource("AbstractImplementsFunction.java");
+    File abstractJar = new File(temporaryFolder.getRoot(), "abstract.jar");
+    jarBuilder.buildJar(abstractJar, sourceFileOne);
+
+    jarBuilder.addToClasspath(abstractJar);
+    File sourceFileTwo = loadTestResource("AnnotatedFunction.java");
+
+    jarBuilder.buildJar(outputJar, sourceFileTwo);
+    Collection<String> functionsFoundInJar = functionScanner.findFunctionsInJar(outputJar);
+    assertThat(functionsFoundInJar).containsExactlyInAnyOrder(
+        "org.apache.geode.management.internal.deployment.AnnotatedFunction");
+  }
+
   private File loadTestResource(String fileName) throws URISyntaxException {
     URL resourceFileURL = this.getClass().getResource(fileName);
     assertThat(resourceFileURL).isNotNull();

http://git-wip-us.apache.org/repos/asf/geode/blob/1a67d462/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractFunction.java b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractFunction.java
deleted file mode 100644
index afc83ab..0000000
--- a/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractFunction.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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.geode.management.internal.deployment;
-
-import org.apache.geode.cache.execute.FunctionContext;
-
-public class AbstractFunction implements Function {
-  public void execute(FunctionContext context) {
-    context.getResultSender().lastResult("ConcreteResult");
-  }
-
-  public static abstract class AbstractImplementsFunction implements Function {
-    public abstract void execute(FunctionContext context);
-  }
-
-  public static class Concrete extends AbstractImplementsFunction {
-    public void execute(FunctionContext context) {
-      context.getResultSender().lastResult("ConcreteResult");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1a67d462/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AnnotatedFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AnnotatedFunction.java b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AnnotatedFunction.java
new file mode 100644
index 0000000..612b498
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AnnotatedFunction.java
@@ -0,0 +1,23 @@
+/*
+ * 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.geode.management.internal.deployment;
+
+import org.apache.geode.cache.execute.FunctionContext;
+
+public class AnnotatedFunction extends AbstractImplementsFunction {
+  public void execute(FunctionContext context) {
+    context.getResultSender().lastResult("AnnotatedFunctionResult");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/1a67d462/geode-junit/src/main/java/org/apache/geode/test/compiler/JarBuilder.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/org/apache/geode/test/compiler/JarBuilder.java b/geode-junit/src/main/java/org/apache/geode/test/compiler/JarBuilder.java
index beea476..db1eb58 100644
--- a/geode-junit/src/main/java/org/apache/geode/test/compiler/JarBuilder.java
+++ b/geode-junit/src/main/java/org/apache/geode/test/compiler/JarBuilder.java
@@ -24,8 +24,6 @@ import java.util.List;
 import java.util.jar.JarEntry;
 import java.util.jar.JarOutputStream;
 
-import org.assertj.core.api.Assertions;
-
 
 /**
  * This class accepts java source code in the format of .java source files or strings containing the
@@ -76,6 +74,14 @@ import org.assertj.core.api.Assertions;
 public class JarBuilder {
   private final JavaCompiler javaCompiler = new JavaCompiler();
 
+  /**
+   * Adds the given jarFile to the classpath that will be used for compilation by the buildJar
+   * methods.
+   */
+  public void addToClasspath(File jarFile) {
+    javaCompiler.addToClasspath(jarFile);
+  }
+
   public void buildJarFromClassNames(File outputJarFile, String... classNames) throws IOException {
     UncompiledSourceCode[] uncompiledSourceCodes = Arrays.stream(classNames)
         .map(UncompiledSourceCode::fromClassName).toArray(UncompiledSourceCode[]::new);

http://git-wip-us.apache.org/repos/asf/geode/blob/1a67d462/geode-junit/src/main/java/org/apache/geode/test/compiler/JavaCompiler.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/org/apache/geode/test/compiler/JavaCompiler.java b/geode-junit/src/main/java/org/apache/geode/test/compiler/JavaCompiler.java
index 8449605..6039e87 100644
--- a/geode-junit/src/main/java/org/apache/geode/test/compiler/JavaCompiler.java
+++ b/geode-junit/src/main/java/org/apache/geode/test/compiler/JavaCompiler.java
@@ -32,10 +32,16 @@ import org.apache.commons.io.FileUtils;
 
 public class JavaCompiler {
   private File tempDir;
+  private String classpath;
 
   public JavaCompiler() {
     this.tempDir = Files.createTempDir();
     tempDir.deleteOnExit();
+    this.classpath = System.getProperty("java.class.path");
+  }
+
+  public void addToClasspath(File jarFile) {
+    classpath += File.pathSeparator + jarFile.getAbsolutePath();
   }
 
   public List<CompiledSourceCode> compile(File... sourceFiles) throws IOException {
@@ -57,8 +63,9 @@ public class JavaCompiler {
     File temporarySourcesDirectory = createSubdirectory(tempDir, "sources");
     File temporaryClassesDirectory = createSubdirectory(tempDir, "classes");
 
-    List<String> options = Stream.of("-d", temporaryClassesDirectory.getAbsolutePath(),
-        "-classpath", System.getProperty("java.class.path")).collect(toList());
+    List<String> options =
+        Stream.of("-d", temporaryClassesDirectory.getAbsolutePath(), "-classpath", classpath)
+            .collect(toList());
 
     try {
       for (UncompiledSourceCode sourceCode : uncompiledSources) {


[2/2] geode git commit: GEODE-3235: Deploy jar registers functions which extend FunctionAdapter

Posted by js...@apache.org.
GEODE-3235: Deploy jar registers functions which extend FunctionAdapter


Project: http://git-wip-us.apache.org/repos/asf/geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/geode/commit/64f33c3e
Tree: http://git-wip-us.apache.org/repos/asf/geode/tree/64f33c3e
Diff: http://git-wip-us.apache.org/repos/asf/geode/diff/64f33c3e

Branch: refs/heads/develop
Commit: 64f33c3e456af775d7ee35b05a67f76cb3a23941
Parents: 82fad64
Author: Jared Stewart <js...@pivotal.io>
Authored: Tue Jul 25 15:32:18 2017 -0700
Committer: Jared Stewart <js...@pivotal.io>
Committed: Thu Aug 17 15:57:59 2017 -0700

----------------------------------------------------------------------
 .../org/apache/geode/internal/DeployedJar.java  |  49 ++++----
 .../internal/deployment/FunctionScanner.java    |  47 ++++++++
 ...loyCommandFunctionRegistrationDUnitTest.java | 118 +++++++++++++++++++
 .../deployment/FunctionScannerTest.java         | 106 +++++++++++++++++
 .../AbstractExtendsFunctionAdapter.java         |  24 ++++
 .../internal/deployment/AbstractFunction.java   |  33 ++++++
 .../deployment/AbstractImplementsFunction.java  |  24 ++++
 ...teExtendsAbstractExtendsFunctionAdapter.java |  23 ++++
 ...ncreteExtendsAbstractImplementsFunction.java |  23 ++++
 .../deployment/ExtendsAbstractFunction.java     |  25 ++++
 .../deployment/ExtendsFunctionAdapter.java      |  25 ++++
 .../internal/deployment/ImplementsFunction.java |  24 ++++
 12 files changed, 494 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java b/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
index 037ef9e..a341ee3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
@@ -14,19 +14,6 @@
  */
 package org.apache.geode.internal;
 
-import io.github.lukehutch.fastclasspathscanner.FastClasspathScanner;
-import io.github.lukehutch.fastclasspathscanner.scanner.ScanResult;
-import org.apache.commons.collections.CollectionUtils;
-import org.apache.geode.cache.CacheClosedException;
-import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.Declarable;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.execute.FunctionService;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.pdx.internal.TypeRegistry;
-import org.apache.logging.log4j.Logger;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
@@ -38,7 +25,6 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.Modifier;
 import java.net.MalformedURLException;
 import java.net.URL;
-import java.net.URLClassLoader;
 import java.nio.file.Files;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
@@ -53,9 +39,22 @@ import java.util.jar.JarInputStream;
 import java.util.regex.Pattern;
 import java.util.stream.Stream;
 
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.cache.CacheClosedException;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.Declarable;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.management.internal.deployment.FunctionScanner;
+import org.apache.geode.pdx.internal.TypeRegistry;
+
 /**
  * ClassLoader for a single JAR file.
- * 
+ *
  * @since GemFire 7.0
  */
 public class DeployedJar {
@@ -123,7 +122,7 @@ public class DeployedJar {
 
   /**
    * Peek into the JAR data and make sure that it is valid JAR content.
-   * 
+   *
    * @param inputStream InputStream containing data to be validated.
    * @return True if the data has JAR content, false otherwise
    */
@@ -149,7 +148,7 @@ public class DeployedJar {
 
   /**
    * Peek into the JAR data and make sure that it is valid JAR content.
-   * 
+   *
    * @param jarBytes Bytes of data to be validated.
    * @return True if the data has JAR content, false otherwise
    */
@@ -171,7 +170,7 @@ public class DeployedJar {
 
     JarInputStream jarInputStream = null;
     try {
-      List<String> functionClasses = findFunctionsInThisJar();
+      Collection<String> functionClasses = findFunctionsInThisJar();
 
       jarInputStream = new JarInputStream(byteArrayInputStream);
       JarEntry jarEntry = jarInputStream.getNextJarEntry();
@@ -259,7 +258,7 @@ public class DeployedJar {
   /**
    * Uses MD5 hashes to determine if the original byte content of this DeployedJar is the same as
    * that past in.
-   * 
+   *
    * @param compareToBytes Bytes to compare the original content to
    * @return True of the MD5 hash is the same o
    */
@@ -281,7 +280,7 @@ public class DeployedJar {
    * Check to see if the class implements the Function interface. If so, it will be registered with
    * FunctionService. Also, if the functions's class was originally declared in a cache.xml file
    * then any properties specified at that time will be reused when re-registering the function.
-   * 
+   *
    * @param clazz Class to check for implementation of the Function class
    * @return A collection of Objects that implement the Function interface.
    */
@@ -333,15 +332,11 @@ public class DeployedJar {
     return registerableFunctions;
   }
 
-  private List<String> findFunctionsInThisJar() throws IOException {
-    URLClassLoader urlClassLoader =
-        new URLClassLoader(new URL[] {this.getFile().getCanonicalFile().toURL()});
-    FastClasspathScanner fastClasspathScanner = new FastClasspathScanner()
-        .removeTemporaryFilesAfterScan(true).overrideClassLoaders(urlClassLoader);
-    ScanResult scanResult = fastClasspathScanner.scan();
-    return scanResult.getNamesOfClassesImplementing(Function.class);
+  protected Collection<String> findFunctionsInThisJar() throws IOException {
+    return new FunctionScanner().findFunctionsInJar(this.file);
   }
 
+
   private Function newFunction(final Class<Function> clazz, final boolean errorOnNoSuchMethod) {
     try {
       final Constructor<Function> constructor = clazz.getConstructor();

http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/main/java/org/apache/geode/management/internal/deployment/FunctionScanner.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/deployment/FunctionScanner.java b/geode-core/src/main/java/org/apache/geode/management/internal/deployment/FunctionScanner.java
new file mode 100644
index 0000000..9b7d6c4
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/deployment/FunctionScanner.java
@@ -0,0 +1,47 @@
+/*
+ * 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.geode.management.internal.deployment;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import io.github.lukehutch.fastclasspathscanner.FastClasspathScanner;
+import io.github.lukehutch.fastclasspathscanner.scanner.ScanResult;
+
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionAdapter;
+
+public class FunctionScanner {
+
+  public Collection<String> findFunctionsInJar(File jarFile) throws IOException {
+    URLClassLoader urlClassLoader =
+        new URLClassLoader(new URL[] {jarFile.getCanonicalFile().toURL()});
+    FastClasspathScanner fastClasspathScanner = new FastClasspathScanner()
+        .removeTemporaryFilesAfterScan(true).overrideClassLoaders(urlClassLoader);
+    ScanResult scanResult = fastClasspathScanner.scan();
+
+    Set<String> functionClasses = new HashSet<>();
+
+    functionClasses.addAll(scanResult.getNamesOfClassesImplementing(Function.class));
+    functionClasses.addAll(scanResult.getNamesOfSubclassesOf(FunctionAdapter.class));
+
+    return functionClasses;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandFunctionRegistrationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandFunctionRegistrationDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandFunctionRegistrationDUnitTest.java
new file mode 100644
index 0000000..6b933bc
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandFunctionRegistrationDUnitTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.geode.management.internal.cli.commands;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.io.Serializable;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.List;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.execute.Execution;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.internal.ClassPathLoader;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.test.compiler.JarBuilder;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+@Category(DistributedTest.class)
+public class DeployCommandFunctionRegistrationDUnitTest implements Serializable {
+  private MemberVM locator;
+  private MemberVM server;
+
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  @Rule
+  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+
+  @Rule
+  public transient GfshShellConnectionRule gfshConnector = new GfshShellConnectionRule();
+
+  @Before
+  public void setup() throws Exception {
+    locator = lsRule.startLocatorVM(0);
+    server = lsRule.startServerVM(1, locator.getPort());
+
+    gfshConnector.connectAndVerify(locator);
+  }
+
+  @Test
+  public void deployImplements() throws Exception {
+    JarBuilder jarBuilder = new JarBuilder();
+    File source = loadTestResource(
+        "/org/apache/geode/management/internal/deployment/ImplementsFunction.java");
+
+    File outputJar = new File(temporaryFolder.getRoot(), "output.jar");
+    jarBuilder.buildJar(outputJar, source);
+
+    gfshConnector.executeAndVerifyCommand("deploy --jar=" + outputJar.getCanonicalPath());
+    server.invoke(() -> assertThatCanLoad(
+        "org.apache.geode.management.internal.deployment.ImplementsFunction"));
+    server.invoke(() -> assertThatFunctionHasVersion(
+        "org.apache.geode.management.internal.deployment.ImplementsFunction",
+        "ImplementsFunctionResult"));
+  }
+
+  @Test
+  public void deployExtends() throws Exception {
+    JarBuilder jarBuilder = new JarBuilder();
+    File source = loadTestResource(
+        "/org/apache/geode/management/internal/deployment/ExtendsFunctionAdapter.java");
+
+    File outputJar = new File(temporaryFolder.getRoot(), "output.jar");
+    jarBuilder.buildJar(outputJar, source);
+
+    gfshConnector.executeAndVerifyCommand("deploy --jar=" + outputJar.getCanonicalPath());
+    server.invoke(() -> assertThatCanLoad(
+        "org.apache.geode.management.internal.deployment.ExtendsFunctionAdapter"));
+    server.invoke(() -> assertThatFunctionHasVersion(
+        "org.apache.geode.management.internal.deployment.ExtendsFunctionAdapter",
+        "ExtendsFunctionAdapterResult"));
+  }
+
+  private File loadTestResource(String fileName) throws URISyntaxException {
+    URL resourceFileURL = this.getClass().getResource(fileName);
+    assertThat(resourceFileURL).isNotNull();
+
+    URI resourceUri = resourceFileURL.toURI();
+    return new File(resourceUri);
+  }
+
+  private void assertThatFunctionHasVersion(String functionId, String version) {
+    GemFireCacheImpl gemFireCache = GemFireCacheImpl.getInstance();
+    DistributedSystem distributedSystem = gemFireCache.getDistributedSystem();
+    Execution execution = FunctionService.onMember(distributedSystem.getDistributedMember());
+    List<String> result = (List<String>) execution.execute(functionId).getResult();
+    assertThat(result.get(0)).isEqualTo(version);
+  }
+
+  private void assertThatCanLoad(String className) throws ClassNotFoundException {
+    assertThat(ClassPathLoader.getLatest().forName(className)).isNotNull();
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/test/java/org/apache/geode/management/internal/deployment/FunctionScannerTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/deployment/FunctionScannerTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/deployment/FunctionScannerTest.java
new file mode 100644
index 0000000..af9ffdf
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/deployment/FunctionScannerTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.geode.management.internal.deployment;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.Collection;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.test.compiler.JarBuilder;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class FunctionScannerTest {
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private JarBuilder jarBuilder;
+  private FunctionScanner functionScanner;
+  private File outputJar;
+
+  @Before
+  public void setup() {
+    jarBuilder = new JarBuilder();
+    functionScanner = new FunctionScanner();
+    outputJar = new File(temporaryFolder.getRoot(), "output.jar");
+  }
+
+  @Test
+  public void implementsFunction() throws Exception {
+    File sourceFileOne = loadTestResource("ImplementsFunction.java");
+
+    jarBuilder.buildJar(outputJar, sourceFileOne);
+
+    Collection<String> functionsFoundInJar = functionScanner.findFunctionsInJar(outputJar);
+    assertThat(functionsFoundInJar)
+        .containsExactly("org.apache.geode.management.internal.deployment.ImplementsFunction");
+  }
+
+  @Test
+  public void extendsFunctionAdapter() throws Exception {
+    File sourceFileOne = loadTestResource("ExtendsFunctionAdapter.java");
+
+    jarBuilder.buildJar(outputJar, sourceFileOne);
+
+    Collection<String> functionsFoundInJar = functionScanner.findFunctionsInJar(outputJar);
+    assertThat(functionsFoundInJar)
+        .containsExactly("org.apache.geode.management.internal.deployment.ExtendsFunctionAdapter");
+  }
+
+  @Test
+  public void testConcreteExtendsAbstractExtendsFunctionAdapter() throws Exception {
+    File sourceFileOne = loadTestResource("AbstractExtendsFunctionAdapter.java");
+    File sourceFileTwo = loadTestResource("ConcreteExtendsAbstractExtendsFunctionAdapter.java");
+
+    jarBuilder.buildJar(outputJar, sourceFileOne, sourceFileTwo);
+
+    Collection<String> functionsFoundInJar = functionScanner.findFunctionsInJar(outputJar);
+    assertThat(functionsFoundInJar).containsExactlyInAnyOrder(
+        "org.apache.geode.management.internal.deployment.ConcreteExtendsAbstractExtendsFunctionAdapter",
+        "org.apache.geode.management.internal.deployment.AbstractExtendsFunctionAdapter");
+  }
+
+  @Test
+  public void testConcreteExtendsAbstractImplementsFunction() throws Exception {
+    File sourceFileOne = loadTestResource("AbstractImplementsFunction.java");
+    File sourceFileTwo = loadTestResource("ConcreteExtendsAbstractImplementsFunction.java");
+
+    jarBuilder.buildJar(outputJar, sourceFileOne, sourceFileTwo);
+
+    Collection<String> functionsFoundInJar = functionScanner.findFunctionsInJar(outputJar);
+    assertThat(functionsFoundInJar).containsExactlyInAnyOrder(
+        "org.apache.geode.management.internal.deployment.ConcreteExtendsAbstractImplementsFunction",
+        "org.apache.geode.management.internal.deployment.AbstractImplementsFunction");
+  }
+
+  private File loadTestResource(String fileName) throws URISyntaxException {
+    URL resourceFileURL = this.getClass().getResource(fileName);
+    assertThat(resourceFileURL).isNotNull();
+
+    URI resourceUri = resourceFileURL.toURI();
+    return new File(resourceUri);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractExtendsFunctionAdapter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractExtendsFunctionAdapter.java b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractExtendsFunctionAdapter.java
new file mode 100644
index 0000000..5bcc22c
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractExtendsFunctionAdapter.java
@@ -0,0 +1,24 @@
+/*
+ * 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.geode.management.internal.deployment;
+
+import org.apache.geode.cache.execute.FunctionAdapter;
+import org.apache.geode.cache.execute.FunctionContext;
+
+public abstract class AbstractExtendsFunctionAdapter extends FunctionAdapter {
+  public void execute(FunctionContext context) {
+    context.getResultSender().lastResult("AbstractExtendsFunctionAdapterResult");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractFunction.java b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractFunction.java
new file mode 100644
index 0000000..afc83ab
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractFunction.java
@@ -0,0 +1,33 @@
+/*
+ * 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.geode.management.internal.deployment;
+
+import org.apache.geode.cache.execute.FunctionContext;
+
+public class AbstractFunction implements Function {
+  public void execute(FunctionContext context) {
+    context.getResultSender().lastResult("ConcreteResult");
+  }
+
+  public static abstract class AbstractImplementsFunction implements Function {
+    public abstract void execute(FunctionContext context);
+  }
+
+  public static class Concrete extends AbstractImplementsFunction {
+    public void execute(FunctionContext context) {
+      context.getResultSender().lastResult("ConcreteResult");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractImplementsFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractImplementsFunction.java b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractImplementsFunction.java
new file mode 100644
index 0000000..a31399d
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/AbstractImplementsFunction.java
@@ -0,0 +1,24 @@
+/*
+ * 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.geode.management.internal.deployment;
+
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+
+public abstract class AbstractImplementsFunction implements Function {
+  public void execute(FunctionContext context) {
+    context.getResultSender().lastResult("AbstractImplementsFunctionResult");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ConcreteExtendsAbstractExtendsFunctionAdapter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ConcreteExtendsAbstractExtendsFunctionAdapter.java b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ConcreteExtendsAbstractExtendsFunctionAdapter.java
new file mode 100644
index 0000000..3515558
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ConcreteExtendsAbstractExtendsFunctionAdapter.java
@@ -0,0 +1,23 @@
+/*
+ * 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.geode.management.internal.deployment;
+
+import org.apache.geode.cache.execute.FunctionContext;
+
+public class ConcreteExtendsAbstractExtendsFunctionAdapter extends AbstractExtendsFunctionAdapter {
+  public void execute(FunctionContext context) {
+    context.getResultSender().lastResult("ConcreteExtendsAbstractExtendsFunctionAdapter");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ConcreteExtendsAbstractImplementsFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ConcreteExtendsAbstractImplementsFunction.java b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ConcreteExtendsAbstractImplementsFunction.java
new file mode 100644
index 0000000..b62f38b
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ConcreteExtendsAbstractImplementsFunction.java
@@ -0,0 +1,23 @@
+/*
+ * 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.geode.management.internal.deployment;
+
+import org.apache.geode.cache.execute.FunctionContext;
+
+public class ConcreteExtendsAbstractImplementsFunction extends AbstractImplementsFunction {
+  public void execute(FunctionContext context) {
+    context.getResultSender().lastResult("ConcreteExtendsAbstractImplementsFunctionResult");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ExtendsAbstractFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ExtendsAbstractFunction.java b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ExtendsAbstractFunction.java
new file mode 100644
index 0000000..cf7c7a2
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ExtendsAbstractFunction.java
@@ -0,0 +1,25 @@
+/*
+ * 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.geode.management.internal.deployment;
+
+
+import org.apache.geode.cache.execute.FunctionAdapter;
+import org.apache.geode.cache.execute.FunctionContext;
+
+public class ExtendsFunctionAdapter extends FunctionAdapter {
+  public void execute(FunctionContext context) {
+    context.getResultSender().lastResult("ExtendsFunctionAdapterResult");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ExtendsFunctionAdapter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ExtendsFunctionAdapter.java b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ExtendsFunctionAdapter.java
new file mode 100644
index 0000000..cf7c7a2
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ExtendsFunctionAdapter.java
@@ -0,0 +1,25 @@
+/*
+ * 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.geode.management.internal.deployment;
+
+
+import org.apache.geode.cache.execute.FunctionAdapter;
+import org.apache.geode.cache.execute.FunctionContext;
+
+public class ExtendsFunctionAdapter extends FunctionAdapter {
+  public void execute(FunctionContext context) {
+    context.getResultSender().lastResult("ExtendsFunctionAdapterResult");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/64f33c3e/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ImplementsFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ImplementsFunction.java b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ImplementsFunction.java
new file mode 100644
index 0000000..c9fef3c
--- /dev/null
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/deployment/ImplementsFunction.java
@@ -0,0 +1,24 @@
+/*
+ * 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.geode.management.internal.deployment;
+
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+
+public class ImplementsFunction implements Function {
+  public void execute(FunctionContext context) {
+    context.getResultSender().lastResult("ImplementsFunctionResult");
+  }
+}