Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 4F89D200D01 for ; Fri, 18 Aug 2017 00:59:52 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 4DF3E16BFF8; Thu, 17 Aug 2017 22:59:52 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id C966D16BFF4 for ; Fri, 18 Aug 2017 00:59:50 +0200 (CEST) Received: (qmail 20738 invoked by uid 500); 17 Aug 2017 22:59:49 -0000 Mailing-List: contact commits-help@geode.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@geode.apache.org Delivered-To: mailing list commits@geode.apache.org Received: (qmail 20708 invoked by uid 99); 17 Aug 2017 22:59:49 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 17 Aug 2017 22:59:49 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id AB02AE964E; Thu, 17 Aug 2017 22:59:48 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jstewart@apache.org To: commits@geode.apache.org Date: Thu, 17 Aug 2017 22:59:49 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [2/2] geode git commit: GEODE-3235: Deploy jar registers functions which extend FunctionAdapter archived-at: Thu, 17 Aug 2017 22:59:52 -0000 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 Authored: Tue Jul 25 15:32:18 2017 -0700 Committer: Jared Stewart 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 functionClasses = findFunctionsInThisJar(); + Collection 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 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 findFunctionsInThisJar() throws IOException { + return new FunctionScanner().findFunctionsInJar(this.file); } + private Function newFunction(final Class clazz, final boolean errorOnNoSuchMethod) { try { final Constructor 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 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 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 result = (List) 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 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 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 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 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"); + } +}