You are viewing a plain text version of this content. The canonical link for it is here.
Posted to s4-commits@incubator.apache.org by mm...@apache.org on 2012/07/19 16:26:24 UTC

[19/32] git commit: Refactored application classloading - got rid of the multiple classloaders scheme, since we only load 1 app per cluster - new implementation explodes S4R in a directory and uses a URLClassLoader for fetching classes and resources from

Refactored application classloading
- got rid of the multiple classloaders scheme, since we only load 1 app per cluster
- new implementation explodes S4R in a directory and uses a URLClassLoader for fetching
classes and resources from that directory, first looking at user classes
- dynamically generated classes are loaded from the same URLClassLoader
- packaging of application does not involve exploding dependencies anymore: dependencies
are simply added to a /lib directory of the S4R archive, whereas application classes (jar)
are added in a /app directory


Project: http://git-wip-us.apache.org/repos/asf/incubator-s4/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-s4/commit/46502896
Tree: http://git-wip-us.apache.org/repos/asf/incubator-s4/tree/46502896
Diff: http://git-wip-us.apache.org/repos/asf/incubator-s4/diff/46502896

Branch: refs/heads/S4-57
Commit: 4650289626aff57c2edbe09c721febe6f11a92c7
Parents: 53ed170
Author: Matthieu Morel <mm...@apache.org>
Authored: Wed Jul 11 12:43:36 2012 +0200
Committer: Matthieu Morel <mm...@apache.org>
Committed: Wed Jul 11 15:53:41 2012 +0200

----------------------------------------------------------------------
 .../java/org/apache/s4/base/util/S4RLoader.java    |   82 ++----------
 .../org/apache/s4/base/util/S4RLoaderFactory.java  |   99 +++++++++++++++
 .../java/org/apache/s4/core/DefaultCoreModule.java |    4 +
 .../src/main/java/org/apache/s4/core/Server.java   |    4 +-
 .../s4/core/gen/OverloadDispatcherGenerator.java   |   30 ++---
 .../s4/deploy/DistributedDeploymentManager.java    |    2 +-
 subprojects/s4-core/src/main/resources/logback.xml |    2 +-
 .../apache/s4/deploy/TestAutomaticDeployment.java  |    4 +
 .../s4/deploy/prodcon/TestProducerConsumer.java    |    2 +-
 .../src/main/resources/templates/build.gradle      |   29 ++++-
 test-apps/consumer-app/build.gradle                |   25 +++-
 .../src/main/java/s4app/ConsumerPE.java            |    2 +-
 test-apps/producer-app/build.gradle                |   25 +++-
 .../src/main/java/s4app/ProducerPE.java            |    2 +-
 test-apps/simple-deployable-app-1/build.gradle     |   26 ++++-
 .../main/java/org/apache/s4/deploy/SimplePE.java   |   39 +++----
 .../src/main/resources/resource.txt                |    1 +
 test-apps/simple-deployable-app-2/build.gradle     |   24 +++-
 test-apps/twitter-adapter/build.gradle             |   25 +++-
 test-apps/twitter-counter/build.gradle             |   25 +++-
 20 files changed, 303 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/subprojects/s4-base/src/main/java/org/apache/s4/base/util/S4RLoader.java
----------------------------------------------------------------------
diff --git a/subprojects/s4-base/src/main/java/org/apache/s4/base/util/S4RLoader.java b/subprojects/s4-base/src/main/java/org/apache/s4/base/util/S4RLoader.java
index 4fd110d..f75a867 100644
--- a/subprojects/s4-base/src/main/java/org/apache/s4/base/util/S4RLoader.java
+++ b/subprojects/s4-base/src/main/java/org/apache/s4/base/util/S4RLoader.java
@@ -1,82 +1,24 @@
 package org.apache.s4.base.util;
 
-import java.util.ArrayList;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.jar.JarEntry;
-import java.util.jar.JarFile;
+import java.net.URL;
+import java.net.URLClassLoader;
 
 /**
- * 
- * CREDITS
- * 
- * <p>
- * The source code for this class was derived from <a href=
- * "http://code.google.com/p/db4o-om/source/browse/trunk/objectmanager-api/src/com/db4o/objectmanager/configuration/MultiClassLoader.java"
- * >this project</a>.
- * 
+ * A classloader that fetches and loads classes and resources from :
+ * <ul>
+ * <li>Application classes in an S4R archive</li>
+ * <li>Application dependencies from an S4R archive</li>
+ * <li>Classes dynamically generated
  * 
  */
-public class S4RLoader extends MultiClassLoader {
-
-    private final JarResources jarResource;
-    private final Map<String, byte[]> generatedClassBytes = new HashMap<String, byte[]>();
-    
-    public S4RLoader(String jarPath) {
-        jarResource = new JarResources(jarPath);
-    }
+public class S4RLoader extends URLClassLoader {
 
-    /**
-     * In order to load dynamically generated classes with the same classloader than 
-     * the one used for loading application classes from an s4r archive, we register these
-     * generated classes and bytecode in this classloader. They can be picked later.
-     * 
-     */
-    public void addGeneratedClassBytes(String className, byte[] classBytes) {
-        generatedClassBytes.put(className, classBytes);
-    }
-    
-    @Override
-    protected byte[] loadClassBytes(String className) {
-        if (generatedClassBytes.containsKey(className)) {
-            // note: no need to keep that data any longer
-            return generatedClassBytes.remove(className);
-        }
-        className = formatClassName(className);
-        return jarResource.getResource(className);
+    public S4RLoader(URL[] urls) {
+        super(urls);
     }
 
-    public List<Class<?>> getClasses(String path) {
-        List<Class<?>> classes = new ArrayList<Class<?>>();
-        try {
-            S4RLoader jarLoader = new S4RLoader(path);
-            JarFile jarFile = new JarFile(path);
-            for (Enumeration<JarEntry> e = jarFile.entries(); e
-                    .hasMoreElements();) {
-                try {
-                    JarEntry entry = e.nextElement();
-                    if (entry.getName().endsWith(".class")) {
-                        String className = entry
-                                .getName()
-                                .substring(0, entry.getName().indexOf(".class"))
-                                .replace("/", ".").replace("\\", ".");
-
-                        Class<?> clazz = jarLoader.loadClass(className);
-                        // clazz.asSubclass(IoAdapter.class);
-                        if (clazz != null) {
-                            classes.add(clazz);
-                        }
-                    }
-                } catch (Exception ex) {
-                }
-            }
-
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-        return classes;
+    public Class<?> loadGeneratedClass(String name, byte[] bytes) {
+        return defineClass(name, bytes, 0, bytes.length);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/subprojects/s4-base/src/main/java/org/apache/s4/base/util/S4RLoaderFactory.java
----------------------------------------------------------------------
diff --git a/subprojects/s4-base/src/main/java/org/apache/s4/base/util/S4RLoaderFactory.java b/subprojects/s4-base/src/main/java/org/apache/s4/base/util/S4RLoaderFactory.java
new file mode 100644
index 0000000..bf613e1
--- /dev/null
+++ b/subprojects/s4-base/src/main/java/org/apache/s4/base/util/S4RLoaderFactory.java
@@ -0,0 +1,99 @@
+package org.apache.s4.base.util;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.io.ByteStreams;
+import com.google.common.io.Closeables;
+import com.google.common.io.Files;
+import com.google.inject.Inject;
+import com.google.inject.name.Named;
+
+/**
+ * Helper class for creating S4RLoader instances for a given S4R file.
+ * 
+ */
+public class S4RLoaderFactory {
+
+    private static Logger logger = LoggerFactory.getLogger(S4RLoaderFactory.class);
+
+    @Inject(optional = true)
+    @Named("s4.tmp.dir")
+    File tmpDir;
+
+    /**
+     * Explodes the s4r archive in a user specified directory through "s4.tmpdir" parameter, and prepares a classloader
+     * that will load classes and resources from, first, the application classes, then the dependencies.
+     * 
+     * Inspired from Hadoop's application classloading implementation (RunJar class).
+     * 
+     * @param s4rPath
+     *            path to s4r
+     * @return classloader that loads resources from the s4r in a predefined order
+     */
+    public S4RLoader createS4RLoader(String s4rPath) {
+        if (tmpDir == null) {
+            tmpDir = Files.createTempDir();
+            tmpDir.deleteOnExit();
+            logger.warn(
+                    "s4.tmp.dir not specified, using temporary directory [{}] for unpacking S4R. You should rather specify a non-temporary directory.",
+                    tmpDir.getAbsolutePath());
+        }
+        JarFile jar = null;
+        try {
+            jar = new JarFile(s4rPath);
+            Enumeration<JarEntry> entries = jar.entries();
+            while (entries.hasMoreElements()) {
+                JarEntry entry = entries.nextElement();
+                if (!entry.isDirectory()) {
+                    File to = new File(tmpDir, entry.getName());
+                    Files.createParentDirs(to);
+                    InputStream is = jar.getInputStream(entry);
+                    OutputStream os = new FileOutputStream(to);
+                    try {
+                        ByteStreams.copy(is, os);
+                    } finally {
+                        Closeables.closeQuietly(is);
+                        Closeables.closeQuietly(os);
+                    }
+                }
+            }
+
+            List<URL> classpath = new ArrayList<URL>();
+            addDirLibsToClassPath(classpath, "/app");
+            addDirLibsToClassPath(classpath, "/lib");
+            classpath.add(new File(tmpDir.getAbsolutePath() + "/").toURI().toURL());
+
+            S4RLoader s4rLoader = new S4RLoader(classpath.toArray(new URL[] {}));
+            return s4rLoader;
+
+        } catch (IOException e) {
+            logger.error("Cannot process S4R [{}]: {}", s4rPath, e.getClass().getName() + "/" + e.getMessage());
+            throw new RuntimeException("Cannot create S4R classloader", e);
+        }
+    }
+
+    private void addDirLibsToClassPath(List<URL> classpath, String dir) throws MalformedURLException {
+        File[] libs = new File(tmpDir, dir).listFiles();
+        if (libs != null) {
+            for (int i = 0; i < libs.length; i++) {
+                if (!libs[i].isDirectory()) {
+                    classpath.add(libs[i].toURI().toURL());
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/subprojects/s4-core/src/main/java/org/apache/s4/core/DefaultCoreModule.java
----------------------------------------------------------------------
diff --git a/subprojects/s4-core/src/main/java/org/apache/s4/core/DefaultCoreModule.java b/subprojects/s4-core/src/main/java/org/apache/s4/core/DefaultCoreModule.java
index 4baf10f..b740a21 100644
--- a/subprojects/s4-core/src/main/java/org/apache/s4/core/DefaultCoreModule.java
+++ b/subprojects/s4-core/src/main/java/org/apache/s4/core/DefaultCoreModule.java
@@ -8,6 +8,7 @@ import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.s4.base.Hasher;
 import org.apache.s4.base.SerializerDeserializer;
+import org.apache.s4.base.util.S4RLoaderFactory;
 import org.apache.s4.comm.DefaultHasher;
 import org.apache.s4.comm.serialize.KryoSerDeser;
 import org.apache.s4.deploy.DeploymentManager;
@@ -56,6 +57,8 @@ public class DefaultCoreModule extends AbstractModule {
         bind(SerializerDeserializer.class).to(KryoSerDeser.class);
 
         bind(DeploymentManager.class).to(DistributedDeploymentManager.class);
+
+        bind(S4RLoaderFactory.class);
     }
 
     private void loadProperties(Binder binder) {
@@ -73,4 +76,5 @@ public class DefaultCoreModule extends AbstractModule {
             e.printStackTrace();
         }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/subprojects/s4-core/src/main/java/org/apache/s4/core/Server.java
----------------------------------------------------------------------
diff --git a/subprojects/s4-core/src/main/java/org/apache/s4/core/Server.java b/subprojects/s4-core/src/main/java/org/apache/s4/core/Server.java
index 44e7b56..56d66c9 100644
--- a/subprojects/s4-core/src/main/java/org/apache/s4/core/Server.java
+++ b/subprojects/s4-core/src/main/java/org/apache/s4/core/Server.java
@@ -8,6 +8,7 @@ import java.util.jar.JarFile;
 
 import org.I0Itec.zkclient.ZkClient;
 import org.apache.s4.base.util.S4RLoader;
+import org.apache.s4.base.util.S4RLoaderFactory;
 import org.apache.s4.comm.topology.AssignmentFromZK;
 import org.apache.s4.comm.topology.ZNRecordSerializer;
 import org.apache.s4.deploy.DeploymentManager;
@@ -78,7 +79,8 @@ public class Server {
         // TODO handle application upgrade
         logger.info("Loading application [{}] from file [{}]", appName, s4r.getAbsolutePath());
 
-        S4RLoader cl = new S4RLoader(s4r.getAbsolutePath());
+        S4RLoaderFactory loaderFactory = injector.getInstance(S4RLoaderFactory.class);
+        S4RLoader cl = loaderFactory.createS4RLoader(s4r.getAbsolutePath());
         try {
             JarFile s4rFile = new JarFile(s4r);
             if (s4rFile.getManifest() == null) {

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/subprojects/s4-core/src/main/java/org/apache/s4/core/gen/OverloadDispatcherGenerator.java
----------------------------------------------------------------------
diff --git a/subprojects/s4-core/src/main/java/org/apache/s4/core/gen/OverloadDispatcherGenerator.java b/subprojects/s4-core/src/main/java/org/apache/s4/core/gen/OverloadDispatcherGenerator.java
index cfaa6c2..e225bde 100644
--- a/subprojects/s4-core/src/main/java/org/apache/s4/core/gen/OverloadDispatcherGenerator.java
+++ b/subprojects/s4-core/src/main/java/org/apache/s4/core/gen/OverloadDispatcherGenerator.java
@@ -38,21 +38,18 @@ import org.slf4j.LoggerFactory;
 import com.google.common.io.Files;
 
 /**
- * This class generates a proxy to enable dispatching of events to methods of
- * processing elements based on the runtime type of the event.
+ * This class generates a proxy to enable dispatching of events to methods of processing elements based on the runtime
+ * type of the event.
  * 
  * <p>
- * When an event is transferred to a processing element, the generated proxy
- * finds the corresponding <code>onEvent</code> method with the event type
- * argument matching the current parameter and calls this method.
+ * When an event is transferred to a processing element, the generated proxy finds the corresponding
+ * <code>onEvent</code> method with the event type argument matching the current parameter and calls this method.
  * </p>
  * <p>
- * If there is no exact match, the closest type in the hierarchy of events is
- * used.
+ * If there is no exact match, the closest type in the hierarchy of events is used.
  * </p>
  * <p>
- * If there is still no match, an error statement is logged and the event is
- * ignored (not processed).
+ * If there is still no match, an error statement is logged and the event is ignored (not processed).
  * </p>
  * 
  */
@@ -268,11 +265,11 @@ public class OverloadDispatcherGenerator {
 
     /**
      * 
-     * Delegates to S4Classloader if it was used to load the PE prototype class, 
-     * by passing to S4Classloader the generated bytecode.
+     * Delegates to S4Classloader if it was used to load the PE prototype class, by passing to S4Classloader the
+     * generated bytecode.
      * 
      * Falls back to parent classloader otherwise.
-     *
+     * 
      */
     private static class OverloadDispatcherClassLoader extends ClassLoader {
 
@@ -285,14 +282,7 @@ public class OverloadDispatcherGenerator {
         public Class<?> loadClassFromBytes(String name, byte[] bytes) {
 
             if (s4AppLoader instanceof S4RLoader) {
-                ((S4RLoader)s4AppLoader).addGeneratedClassBytes(name, bytes);
-                try {
-                    return s4AppLoader.loadClass(name);
-                } catch (ClassNotFoundException e) {
-                    // TODO throw a runtime exception?
-                    e.printStackTrace();
-                    return null;
-                }
+                return ((S4RLoader) s4AppLoader).loadGeneratedClass(name, bytes);
             } else {
                 try {
                     return this.loadClass(name);

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/subprojects/s4-core/src/main/java/org/apache/s4/deploy/DistributedDeploymentManager.java
----------------------------------------------------------------------
diff --git a/subprojects/s4-core/src/main/java/org/apache/s4/deploy/DistributedDeploymentManager.java b/subprojects/s4-core/src/main/java/org/apache/s4/deploy/DistributedDeploymentManager.java
index 979cc5d..af94405 100644
--- a/subprojects/s4-core/src/main/java/org/apache/s4/deploy/DistributedDeploymentManager.java
+++ b/subprojects/s4-core/src/main/java/org/apache/s4/deploy/DistributedDeploymentManager.java
@@ -110,12 +110,12 @@ public class DistributedDeploymentManager implements DeploymentManager {
             App loaded = server.loadApp(localS4RFileCopy, appName);
             if (loaded != null) {
                 logger.info("Successfully installed application {}", appName);
+                // TODO sync with other nodes? (e.g. wait for other apps deployed before starting?
                 server.startApp(loaded, appName, clusterName);
             } else {
                 throw new DeploymentFailedException("Cannot deploy application [" + appName + "] from URI ["
                         + uri.toString() + "] : cannot start application");
             }
-            // TODO sync with other nodes? (e.g. wait for other apps deployed before starting?
 
         } catch (URISyntaxException e) {
             logger.error("Cannot deploy app {} : invalid uri for fetching s4r archive {} : {} ", new String[] {

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/subprojects/s4-core/src/main/resources/logback.xml
----------------------------------------------------------------------
diff --git a/subprojects/s4-core/src/main/resources/logback.xml b/subprojects/s4-core/src/main/resources/logback.xml
index 6dac308..ea8c85a 100644
--- a/subprojects/s4-core/src/main/resources/logback.xml
+++ b/subprojects/s4-core/src/main/resources/logback.xml
@@ -8,7 +8,7 @@
     </encoder>
   </appender>
 
-  <root level="trace">
+  <root level="debug">
     <appender-ref ref="STDOUT" />
   </root>
 </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/subprojects/s4-core/src/test/java/org/apache/s4/deploy/TestAutomaticDeployment.java
----------------------------------------------------------------------
diff --git a/subprojects/s4-core/src/test/java/org/apache/s4/deploy/TestAutomaticDeployment.java b/subprojects/s4-core/src/test/java/org/apache/s4/deploy/TestAutomaticDeployment.java
index 495c1ad..108a144 100644
--- a/subprojects/s4-core/src/test/java/org/apache/s4/deploy/TestAutomaticDeployment.java
+++ b/subprojects/s4-core/src/test/java/org/apache/s4/deploy/TestAutomaticDeployment.java
@@ -136,6 +136,10 @@ public class TestAutomaticDeployment {
 
         assertDeployment("http://localhost:8080/s4/" + s4rToDeploy.getName());
 
+        // check resource loading (we use a zkclient without custom serializer)
+        ZkClient client2 = new ZkClient("localhost:" + CommTestUtils.ZK_PORT);
+        Assert.assertEquals("Salut!", client2.readData("/resourceData"));
+
     }
 
     private void initializeS4Node() throws ConfigurationException, IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/subprojects/s4-core/src/test/java/org/apache/s4/deploy/prodcon/TestProducerConsumer.java
----------------------------------------------------------------------
diff --git a/subprojects/s4-core/src/test/java/org/apache/s4/deploy/prodcon/TestProducerConsumer.java b/subprojects/s4-core/src/test/java/org/apache/s4/deploy/prodcon/TestProducerConsumer.java
index 52792b1..935a3cc 100644
--- a/subprojects/s4-core/src/test/java/org/apache/s4/deploy/prodcon/TestProducerConsumer.java
+++ b/subprojects/s4-core/src/test/java/org/apache/s4/deploy/prodcon/TestProducerConsumer.java
@@ -130,7 +130,7 @@ public class TestProducerConsumer {
         CountDownLatch signalConsumptionComplete = new CountDownLatch(1);
         CommTestUtils.watchAndSignalCreation("/1000TicksReceived", signalConsumptionComplete,
                 CommTestUtils.createZkClient());
-        Assert.assertTrue(signalConsumptionComplete.await(20, TimeUnit.SECONDS));
+        Assert.assertTrue(signalConsumptionComplete.await(40, TimeUnit.SECONDS));
 
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/subprojects/s4-tools/src/main/resources/templates/build.gradle
----------------------------------------------------------------------
diff --git a/subprojects/s4-tools/src/main/resources/templates/build.gradle b/subprojects/s4-tools/src/main/resources/templates/build.gradle
index a1e50e3..a55a207 100644
--- a/subprojects/s4-tools/src/main/resources/templates/build.gradle
+++ b/subprojects/s4-tools/src/main/resources/templates/build.gradle
@@ -92,17 +92,34 @@ manifest.mainAttributes(
 
 project.ext["appDependencies"] = ( configurations.compile )
 
+// external dependencies will be available in the /lib directory of the s4r
+task copyDependenciesToLib(type: Copy) {
+    into project.libsDir.path+"/lib"
+    from configurations.runtime
+}
+
+// app jar will be available from the /app directory of the s4r
+task buildProjectJar() {
+	dependsOn jar {
+		destinationDir file(project.libsDir.path + "/app")
+		from sourceSets.main.output
+	}
+}
+
+
+
 /* This task will extract all the class files and create a fat jar. We set the manifest and the extension to make it an S4 archive file. */
-// TODO: exclude schenma files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude schema files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude s4 platform jars
 task s4r(type: Jar) {
-
-   if (rootProject.hasProperty("appName")) {
+	if (rootProject.hasProperty("appName")) {
        archiveName= "$appName"+".s4r"
    }
-   dependsOn jar
-   from { appDependencies.collect { it.isDirectory() ? it : zipTree(it) } }
-   from { configurations.archives.allArtifacts.files.collect { zipTree(it) } }
+   dependsOn cleanCopyDependenciesToLib, copyDependenciesToLib, cleanBuildProjectJar, buildProjectJar
+   from { project.libsDir }
    manifest = project.manifest
+   extension = 's4r'
+   exclude '*.s4r'
 
    // check -PappClassName, need to refer to rootProject to check property, see http://issues.gradle.org/browse/GRADLE-1826
     if (!rootProject.hasProperty('appClass') || !"$appClass") {

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/test-apps/consumer-app/build.gradle
----------------------------------------------------------------------
diff --git a/test-apps/consumer-app/build.gradle b/test-apps/consumer-app/build.gradle
index 3bfe72f..e2fec49 100644
--- a/test-apps/consumer-app/build.gradle
+++ b/test-apps/consumer-app/build.gradle
@@ -133,14 +133,31 @@ manifest.mainAttributes(
 
 project.ext["appDependencies"] = ( configurations.compile )
 
+// external dependencies will be available in the /lib directory of the s4r
+task copyDependenciesToLib(type: Copy) {
+    into project.libsDir.path+"/lib"
+    from configurations.runtime
+}
+
+// app jar will be available from the /app directory of the s4r
+task buildProjectJar() {
+	dependsOn jar {
+		destinationDir file(project.libsDir.path + "/app")
+		from sourceSets.main.output
+	}
+}
+
+
+
 /* This task will extract all the class files and create a fat jar. We set the manifest and the extension to make it an S4 archive file. */
-// TODO: exclude schenma files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude schema files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude s4 platform jars
 task s4r(type: Jar) {
-   dependsOn jar
-   from { appDependencies.collect { it.isDirectory() ? it : zipTree(it) } }
-   from { configurations.archives.allArtifacts.files.collect { it.isDirectory() ? it : zipTree(it) } }
+   dependsOn cleanCopyDependenciesToLib, copyDependenciesToLib, cleanBuildProjectJar, buildProjectJar
+   from { project.libsDir }
    manifest = project.manifest
    extension = 's4r'
+   exclude '*.s4r'
 
    /* Set class name in manifest. Parse source files until we find a class that extends App.
     * Get fully qualified Java class name and set attribute in Manifest.

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/test-apps/consumer-app/src/main/java/s4app/ConsumerPE.java
----------------------------------------------------------------------
diff --git a/test-apps/consumer-app/src/main/java/s4app/ConsumerPE.java b/test-apps/consumer-app/src/main/java/s4app/ConsumerPE.java
index d0b1577..1cc7ed0 100644
--- a/test-apps/consumer-app/src/main/java/s4app/ConsumerPE.java
+++ b/test-apps/consumer-app/src/main/java/s4app/ConsumerPE.java
@@ -19,7 +19,7 @@ public class ConsumerPE extends ProcessingElement {
 
     public void onEvent(Event event) {
         eventCount++;
-        logger.info(
+        logger.trace(
                 "Received event with tick {} and time {} for event # {}",
                 new String[] { String.valueOf(event.get("tick", Long.class)), String.valueOf(event.getTime()),
                         String.valueOf(eventCount) });

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/test-apps/producer-app/build.gradle
----------------------------------------------------------------------
diff --git a/test-apps/producer-app/build.gradle b/test-apps/producer-app/build.gradle
index d71f75f..e95434d 100644
--- a/test-apps/producer-app/build.gradle
+++ b/test-apps/producer-app/build.gradle
@@ -133,14 +133,31 @@ manifest.mainAttributes(
 
 project.ext["appDependencies"] = ( configurations.compile )
 
+// external dependencies will be available in the /lib directory of the s4r
+task copyDependenciesToLib(type: Copy) {
+    into project.libsDir.path+"/lib"
+    from configurations.runtime
+}
+
+// app jar will be available from the /app directory of the s4r
+task buildProjectJar() {
+	dependsOn jar {
+		destinationDir file(project.libsDir.path + "/app")
+		from sourceSets.main.output
+	}
+}
+
+
+
 /* This task will extract all the class files and create a fat jar. We set the manifest and the extension to make it an S4 archive file. */
-// TODO: exclude schenma files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude schema files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude s4 platform jars
 task s4r(type: Jar) {
-   dependsOn jar
-   from { appDependencies.collect { it.isDirectory() ? it : zipTree(it) } }
-   from { configurations.archives.allArtifacts.files.collect { it.isDirectory() ? it : zipTree(it) } }
+   dependsOn cleanCopyDependenciesToLib, copyDependenciesToLib, cleanBuildProjectJar, buildProjectJar
+   from { project.libsDir }
    manifest = project.manifest
    extension = 's4r'
+   exclude '*.s4r'
 
    /* Set class name in manifest. Parse source files until we find a class that extends App.
     * Get fully qualified Java class name and set attribute in Manifest.

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/test-apps/producer-app/src/main/java/s4app/ProducerPE.java
----------------------------------------------------------------------
diff --git a/test-apps/producer-app/src/main/java/s4app/ProducerPE.java b/test-apps/producer-app/src/main/java/s4app/ProducerPE.java
index 7812bc6..57cc7f3 100644
--- a/test-apps/producer-app/src/main/java/s4app/ProducerPE.java
+++ b/test-apps/producer-app/src/main/java/s4app/ProducerPE.java
@@ -31,7 +31,7 @@ public class ProducerPE extends ProcessingElement {
             Event event = new Event();
             event.put("tick", Long.class, tick++);
 
-            logger.info("Sending event with tick {} and time {}.", event.get("tick", Long.class), event.getTime());
+            logger.trace("Sending event with tick {} and time {}.", event.get("tick", Long.class), event.getTime());
             for (int i = 0; i < targetStreams.length; i++) {
                 targetStreams[i].put(event);
             }

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/test-apps/simple-deployable-app-1/build.gradle
----------------------------------------------------------------------
diff --git a/test-apps/simple-deployable-app-1/build.gradle b/test-apps/simple-deployable-app-1/build.gradle
index 6933b58..9ad3970 100644
--- a/test-apps/simple-deployable-app-1/build.gradle
+++ b/test-apps/simple-deployable-app-1/build.gradle
@@ -133,14 +133,32 @@ manifest.mainAttributes(
 
 project.ext["appDependencies"] = ( configurations.compile )
 
+
+// external dependencies will be available in the /lib directory of the s4r
+task copyDependenciesToLib(type: Copy) {
+    into project.libsDir.path+"/lib"
+    from configurations.runtime
+}
+
+// app jar will be available from the /app directory of the s4r
+task buildProjectJar() {
+	dependsOn jar {
+		destinationDir file(project.libsDir.path + "/app")
+		from sourceSets.main.output
+	}
+}
+
+
+
 /* This task will extract all the class files and create a fat jar. We set the manifest and the extension to make it an S4 archive file. */
-// TODO: exclude schenma files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude schema files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude s4 platform jars
 task s4r(type: Jar) {
-   dependsOn jar
-   from { appDependencies.collect { it.isDirectory() ? it : zipTree(it) } }
-   from { configurations.archives.allArtifacts.files.collect { it.isDirectory() ? it : zipTree(it) } }
+   dependsOn cleanCopyDependenciesToLib, copyDependenciesToLib, cleanBuildProjectJar, buildProjectJar
+   from { project.libsDir }
    manifest = project.manifest
    extension = 's4r'
+   exclude '*.s4r'
 
    /* Set class name in manifest. Parse source files until we find a class that extends App.
     * Get fully qualified Java class name and set attribute in Manifest.

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/test-apps/simple-deployable-app-1/src/main/java/org/apache/s4/deploy/SimplePE.java
----------------------------------------------------------------------
diff --git a/test-apps/simple-deployable-app-1/src/main/java/org/apache/s4/deploy/SimplePE.java b/test-apps/simple-deployable-app-1/src/main/java/org/apache/s4/deploy/SimplePE.java
index 8cb5843..23fb9cb 100644
--- a/test-apps/simple-deployable-app-1/src/main/java/org/apache/s4/deploy/SimplePE.java
+++ b/test-apps/simple-deployable-app-1/src/main/java/org/apache/s4/deploy/SimplePE.java
@@ -2,19 +2,19 @@ package org.apache.s4.deploy;
 
 import java.io.IOException;
 
+import org.apache.s4.comm.topology.ZkClient;
 import org.apache.s4.core.App;
 import org.apache.s4.core.ProcessingElement;
 import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.ZooKeeper;
+import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class SimplePE extends ProcessingElement implements Watcher {
+import com.google.common.io.ByteStreams;
 
-    private ZooKeeper zk;
+public class SimplePE extends ProcessingElement {
+
+    private static Logger logger = LoggerFactory.getLogger(SimplePE.class);
+    private ZkClient zk;
 
     public SimplePE() {
     }
@@ -26,12 +26,14 @@ public class SimplePE extends ProcessingElement implements Watcher {
     public void onEvent(org.apache.s4.base.Event event) {
         try {
             LoggerFactory.getLogger(getClass()).debug("processing envent {}", event.get("line"));
-            zk.create("/onEvent@" + event.get("line"), new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            // test s4r resource access
+            zk.create("/resourceData",
+                    new String(ByteStreams.toByteArray(getClass().getResourceAsStream("/resource.txt"))),
+                    CreateMode.PERSISTENT);
+            // test event processing
+            zk.create("/onEvent@" + event.get("line"), new byte[0], CreateMode.PERSISTENT);
             zk.close();
-        } catch (KeeperException e) {
-            // TODO Auto-generated catch block
-            e.printStackTrace();
-        } catch (InterruptedException e) {
+        } catch (IOException e) {
             // TODO Auto-generated catch block
             e.printStackTrace();
         }
@@ -40,24 +42,13 @@ public class SimplePE extends ProcessingElement implements Watcher {
     @Override
     protected void onCreate() {
         if (zk == null) {
-            try {
-                zk = new ZooKeeper("localhost:" + 2181, 4000, this);
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+            zk = new ZkClient("localhost:" + 2181);
         }
 
     }
 
     @Override
     protected void onRemove() {
-        // TODO Auto-generated method stub
-
     }
 
-    @Override
-    public void process(WatchedEvent event) {
-        // TODO Auto-generated method stub
-
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/test-apps/simple-deployable-app-1/src/main/resources/resource.txt
----------------------------------------------------------------------
diff --git a/test-apps/simple-deployable-app-1/src/main/resources/resource.txt b/test-apps/simple-deployable-app-1/src/main/resources/resource.txt
new file mode 100644
index 0000000..7c8b9c8
--- /dev/null
+++ b/test-apps/simple-deployable-app-1/src/main/resources/resource.txt
@@ -0,0 +1 @@
+Salut!
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/test-apps/simple-deployable-app-2/build.gradle
----------------------------------------------------------------------
diff --git a/test-apps/simple-deployable-app-2/build.gradle b/test-apps/simple-deployable-app-2/build.gradle
index 6933b58..38e8eb7 100644
--- a/test-apps/simple-deployable-app-2/build.gradle
+++ b/test-apps/simple-deployable-app-2/build.gradle
@@ -133,12 +133,30 @@ manifest.mainAttributes(
 
 project.ext["appDependencies"] = ( configurations.compile )
 
+
+
+
+
+
+task copyDependenciesToLib(type: Copy) {
+    into project.libsDir.path+"/lib"
+    from configurations.runtime
+}
+
+task buildProjectJar() {
+	dependsOn jar {
+		destinationDir file(project.libsDir.path + "/app")
+		from sourceSets.main.output
+	}
+}
+
+
+
 /* This task will extract all the class files and create a fat jar. We set the manifest and the extension to make it an S4 archive file. */
 // TODO: exclude schenma files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
 task s4r(type: Jar) {
-   dependsOn jar
-   from { appDependencies.collect { it.isDirectory() ? it : zipTree(it) } }
-   from { configurations.archives.allArtifacts.files.collect { it.isDirectory() ? it : zipTree(it) } }
+   dependsOn cleanCopyDependenciesToLib, copyDependenciesToLib, cleanBuildProjectJar, buildProjectJar
+   from { project.libsDir }
    manifest = project.manifest
    extension = 's4r'
 

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/test-apps/twitter-adapter/build.gradle
----------------------------------------------------------------------
diff --git a/test-apps/twitter-adapter/build.gradle b/test-apps/twitter-adapter/build.gradle
index 06c5a29..7157e6f 100644
--- a/test-apps/twitter-adapter/build.gradle
+++ b/test-apps/twitter-adapter/build.gradle
@@ -112,14 +112,31 @@ manifest.mainAttributes(
 
 project.ext["appDependencies"] = ( configurations.compile )
 
+// external dependencies will be available in the /lib directory of the s4r
+task copyDependenciesToLib(type: Copy) {
+    into project.libsDir.path+"/lib"
+    from configurations.runtime
+}
+
+// app jar will be available from the /app directory of the s4r
+task buildProjectJar() {
+	dependsOn jar {
+		destinationDir file(project.libsDir.path + "/app")
+		from sourceSets.main.output
+	}
+}
+
+
+
 /* This task will extract all the class files and create a fat jar. We set the manifest and the extension to make it an S4 archive file. */
-// TODO: exclude schenma files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude schema files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude s4 platform jars
 task s4r(type: Jar) {
-   dependsOn jar
-   from { appDependencies.collect { it.isDirectory() ? it : zipTree(it) } }
-   from { configurations.archives.allArtifacts.files.collect { it.isDirectory() ? it : zipTree(it) } }
+   dependsOn cleanCopyDependenciesToLib, copyDependenciesToLib, cleanBuildProjectJar, buildProjectJar
+   from { project.libsDir }
    manifest = project.manifest
    extension = 's4r'
+   exclude '*.s4r'
 
    /* Set class name in manifest. Parse source files until we find a class that extends App.
     * Get fully qualified Java class name and set attribute in Manifest.

http://git-wip-us.apache.org/repos/asf/incubator-s4/blob/46502896/test-apps/twitter-counter/build.gradle
----------------------------------------------------------------------
diff --git a/test-apps/twitter-counter/build.gradle b/test-apps/twitter-counter/build.gradle
index 6fd625d..c6b7469 100644
--- a/test-apps/twitter-counter/build.gradle
+++ b/test-apps/twitter-counter/build.gradle
@@ -108,14 +108,31 @@ manifest.mainAttributes(
 
 project.ext["appDependencies"] = ( configurations.compile )
 
+// external dependencies will be available in the /lib directory of the s4r
+task copyDependenciesToLib(type: Copy) {
+    into project.libsDir.path+"/lib"
+    from configurations.runtime
+}
+
+// app jar will be available from the /app directory of the s4r
+task buildProjectJar() {
+	dependsOn jar {
+		destinationDir file(project.libsDir.path + "/app")
+		from sourceSets.main.output
+	}
+}
+
+
+
 /* This task will extract all the class files and create a fat jar. We set the manifest and the extension to make it an S4 archive file. */
-// TODO: exclude schenma files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude schema files as needed (not critical) see: http://forums.gradle.org/gradle/topics/using_gradle_to_fat_jar_a_spring_project
+// TODO: exclude s4 platform jars
 task s4r(type: Jar) {
-   dependsOn jar
-   from { appDependencies.collect { it.isDirectory() ? it : zipTree(it) } }
-   from { configurations.archives.allArtifacts.files.collect { zipTree(it) } }
+   dependsOn cleanCopyDependenciesToLib, copyDependenciesToLib, cleanBuildProjectJar, buildProjectJar
+   from { project.libsDir }
    manifest = project.manifest
    extension = 's4r'
+   exclude '*.s4r'
 
    /* Set class name in manifest. Parse source files until we find a class that extends App.
     * Get fully qualified Java class name and set attribute in Manifest.