You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@twill.apache.org by ch...@apache.org on 2014/04/22 08:05:58 UTC

[01/50] [abbrv] git commit: Improve service discovery API to allow push model for getting endpoint changes.

Repository: incubator-twill
Updated Branches:
  refs/heads/site 899986226 -> 2e52a947c (forced update)


Improve service discovery API to allow push model for getting endpoint changes.

This major change to to have the DiscoveryServiceClient.discovery() method to return
a ServiceDiscovered object, which allows caller to set listener to receives changes
service endpoint. The ServiceDiscovered object by itself extends from Iterable to maintain
existing pull model and backward compatibility.

Signed-off-by: Albert Shau <al...@continuuity.com>


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

Branch: refs/heads/site
Commit: b0dd8e241b55532c0a8c5d18f5bccf6395e3b2ea
Parents: 1380f21
Author: Terence Yim <te...@continuuity.com>
Authored: Thu Jan 16 12:59:43 2014 -0800
Committer: Albert Shau <al...@continuuity.com>
Committed: Thu Jan 16 14:09:46 2014 -0800

----------------------------------------------------------------------
 .../org/apache/twill/api/TwillController.java   |   6 +-
 .../twill/internal/AbstractTwillController.java |   4 +-
 .../twill/discovery/DiscoveryServiceClient.java |   6 +-
 .../twill/discovery/ServiceDiscovered.java      |  71 +++++
 .../discovery/DefaultServiceDiscovered.java     | 159 +++++++++++
 .../twill/discovery/DiscoverableAdapter.java    | 102 +++++++
 .../discovery/InMemoryDiscoveryService.java     |  63 +++--
 .../twill/discovery/ZKDiscoveryService.java     | 111 +-------
 .../discovery/DiscoveryServiceTestBase.java     | 278 +++++++++++++++++++
 .../discovery/InMemoryDiscoveryServiceTest.java |  44 +--
 .../twill/discovery/ZKDiscoveryServiceTest.java | 130 +--------
 11 files changed, 690 insertions(+), 284 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/b0dd8e24/twill-api/src/main/java/org/apache/twill/api/TwillController.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/TwillController.java b/twill-api/src/main/java/org/apache/twill/api/TwillController.java
index f31d3f9..a5906f4 100644
--- a/twill-api/src/main/java/org/apache/twill/api/TwillController.java
+++ b/twill-api/src/main/java/org/apache/twill/api/TwillController.java
@@ -19,6 +19,7 @@ package org.apache.twill.api;
 
 import org.apache.twill.api.logging.LogHandler;
 import org.apache.twill.discovery.Discoverable;
+import org.apache.twill.discovery.ServiceDiscovered;
 import com.google.common.util.concurrent.ListenableFuture;
 
 /**
@@ -35,10 +36,9 @@ public interface TwillController extends ServiceController {
   /**
    * Discovers the set of {@link Discoverable} endpoints that provides service for the given service name.
    * @param serviceName Name of the service to discovery.
-   * @return An {@link Iterable} that gives set of latest {@link Discoverable} every time when
-   *         {@link Iterable#iterator()}} is invoked.
+   * @return A {@link org.apache.twill.discovery.ServiceDiscovered} object representing the result.
    */
-  Iterable<Discoverable> discoverService(String serviceName);
+  ServiceDiscovered discoverService(String serviceName);
 
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/b0dd8e24/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java b/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
index 5806f9d..97e0a8f 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
@@ -21,8 +21,8 @@ import org.apache.twill.api.RunId;
 import org.apache.twill.api.TwillController;
 import org.apache.twill.api.logging.LogEntry;
 import org.apache.twill.api.logging.LogHandler;
-import org.apache.twill.discovery.Discoverable;
 import org.apache.twill.discovery.DiscoveryServiceClient;
+import org.apache.twill.discovery.ServiceDiscovered;
 import org.apache.twill.discovery.ZKDiscoveryService;
 import org.apache.twill.internal.json.StackTraceElementCodec;
 import org.apache.twill.internal.kafka.client.SimpleKafkaClient;
@@ -97,7 +97,7 @@ public abstract class AbstractTwillController extends AbstractZKServiceControlle
   }
 
   @Override
-  public final Iterable<Discoverable> discoverService(String serviceName) {
+  public final ServiceDiscovered discoverService(String serviceName) {
     return discoveryServiceClient.discover(serviceName);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/b0dd8e24/twill-discovery-api/src/main/java/org/apache/twill/discovery/DiscoveryServiceClient.java
----------------------------------------------------------------------
diff --git a/twill-discovery-api/src/main/java/org/apache/twill/discovery/DiscoveryServiceClient.java b/twill-discovery-api/src/main/java/org/apache/twill/discovery/DiscoveryServiceClient.java
index 89cf269..a58c83d 100644
--- a/twill-discovery-api/src/main/java/org/apache/twill/discovery/DiscoveryServiceClient.java
+++ b/twill-discovery-api/src/main/java/org/apache/twill/discovery/DiscoveryServiceClient.java
@@ -26,9 +26,7 @@ public interface DiscoveryServiceClient {
    * Retrieves a list of {@link Discoverable} for the a service with the given name.
    *
    * @param name Name of the service
-   * @return A live {@link Iterable} that on each call to {@link Iterable#iterator()} returns
-   *         an {@link java.util.Iterator Iterator} that reflects the latest set of
-   *         available {@link Discoverable} services.
+   * @return A {@link org.apache.twill.discovery.ServiceDiscovered} object representing the result.
    */
-  Iterable<Discoverable> discover(String name);
+  ServiceDiscovered discover(String name);
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/b0dd8e24/twill-discovery-api/src/main/java/org/apache/twill/discovery/ServiceDiscovered.java
----------------------------------------------------------------------
diff --git a/twill-discovery-api/src/main/java/org/apache/twill/discovery/ServiceDiscovered.java b/twill-discovery-api/src/main/java/org/apache/twill/discovery/ServiceDiscovered.java
new file mode 100644
index 0000000..5de15f0
--- /dev/null
+++ b/twill-discovery-api/src/main/java/org/apache/twill/discovery/ServiceDiscovered.java
@@ -0,0 +1,71 @@
+/*
+ * 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.twill.discovery;
+
+import org.apache.twill.common.Cancellable;
+
+import java.util.concurrent.Executor;
+
+/**
+ * Represents the result of service discovery. It extends from
+ * {@link Iterable} that gives set of latest {@link Discoverable} every time when
+ * {@link #iterator()} is invoked.
+ */
+public interface ServiceDiscovered extends Iterable<Discoverable> {
+
+  /**
+   * Represents a callback for watching changes in the discovery list.
+   *
+   * @see #watchChanges(ServiceDiscovered.ChangeListener, java.util.concurrent.Executor)
+   */
+  interface ChangeListener {
+
+    /**
+     * This method will be invoked when the discovery list changed.
+     *
+     * @param serviceDiscovered The {@link ServiceDiscovered} that this listener is attached to.
+     */
+    void onChange(ServiceDiscovered serviceDiscovered);
+  }
+
+  /**
+   * Returns the name of the service being discovered.
+   *
+   * @return Name of the service.
+   */
+  String getName();
+
+  /**
+   * Registers a {@link ChangeListener} to watch for changes in the discovery list.
+   * The {@link ChangeListener#onChange(ServiceDiscovered)} method will be triggered when start watching,
+   * and on every subsequent changes in the discovery list.
+   *
+   * @param listener A {@link ChangeListener} to watch for changes.
+   * @param executor A {@link Executor} for issuing call to the given listener.
+   * @return A {@link Cancellable} to cancel the watch.
+   */
+  Cancellable watchChanges(ChangeListener listener, Executor executor);
+
+  /**
+   * Checks if the given discoverable contains in the current discovery list.
+   *
+   * @param discoverable The {@link Discoverable} to check for.
+   * @return {@code true} if it exists, {@code false} otherwise.
+   */
+  boolean contains(Discoverable discoverable);
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/b0dd8e24/twill-discovery-core/src/main/java/org/apache/twill/discovery/DefaultServiceDiscovered.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/main/java/org/apache/twill/discovery/DefaultServiceDiscovered.java b/twill-discovery-core/src/main/java/org/apache/twill/discovery/DefaultServiceDiscovered.java
new file mode 100644
index 0000000..49fb641
--- /dev/null
+++ b/twill-discovery-core/src/main/java/org/apache/twill/discovery/DefaultServiceDiscovered.java
@@ -0,0 +1,159 @@
+/*
+ * 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.twill.discovery;
+
+import org.apache.twill.common.Cancellable;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * A package private class for implementing {@link ServiceDiscovered}.
+ */
+final class DefaultServiceDiscovered implements ServiceDiscovered {
+
+  private final String name;
+  private final AtomicReference<Set<Discoverable>> discoverables;
+  private final List<ListenerCaller> listenerCallers;
+  private final ReadWriteLock callerLock;
+
+  DefaultServiceDiscovered(String name) {
+    this.name = name;
+    this.discoverables = new AtomicReference<Set<Discoverable>>(ImmutableSet.<Discoverable>of());
+    this.listenerCallers = Lists.newLinkedList();
+    this.callerLock = new ReentrantReadWriteLock();
+  }
+
+  void setDiscoverables(Set<Discoverable> discoverables) {
+    this.discoverables.set(discoverables);
+
+    // Collect all listeners with a read lock to the listener list.
+    List<ListenerCaller> callers = Lists.newArrayList();
+    Lock readLock = callerLock.readLock();
+    readLock.lock();
+    try {
+      callers.addAll(listenerCallers);
+    } finally {
+      readLock.unlock();
+    }
+
+    // Invoke listeners.
+    for (ListenerCaller caller : callers) {
+      caller.invoke();
+    }
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public Cancellable watchChanges(ChangeListener listener, Executor executor) {
+    ListenerCaller caller = new ListenerCaller(listener, executor);
+
+    // Add the new listener with a write lock.
+    Lock writeLock = callerLock.writeLock();
+    writeLock.lock();
+    try {
+      listenerCallers.add(caller);
+    } finally {
+      writeLock.unlock();
+    }
+
+    // Invoke listener for the first time.
+    // Race would happen between this method and the setDiscoverables() method, but it's ok as the contract of
+    // adding a new listener is that onChange will be called at least once. The actual changes is already
+    // reflected by the atomic reference "discoverables", hence it's consistent.
+    caller.invoke();
+    return caller;
+  }
+
+  @Override
+  public boolean contains(Discoverable discoverable) {
+    // If the name doesn't match, it shouldn't be in the list.
+    if (!discoverable.getName().equals(name)) {
+      return false;
+    }
+
+    // Wrap it if necessary for hashCode/equals comparison.
+    Discoverable target = discoverable;
+    if (!(target instanceof DiscoverableWrapper)) {
+      target = new DiscoverableWrapper(target);
+    }
+
+    return discoverables.get().contains(target);
+  }
+
+  @Override
+  public Iterator<Discoverable> iterator() {
+    return discoverables.get().iterator();
+  }
+
+  /**
+   * Private helper class for invoking the change listener from an executor.
+   * It also responsible to remove itself from the listener list.
+   */
+  private final class ListenerCaller implements Runnable, Cancellable {
+
+    private final ChangeListener listener;
+    private final Executor executor;
+    private final AtomicBoolean cancelled;
+
+    private ListenerCaller(ChangeListener listener, Executor executor) {
+      this.listener = listener;
+      this.executor = executor;
+      this.cancelled = new AtomicBoolean(false);
+    }
+
+    void invoke() {
+      if (!cancelled.get()) {
+        executor.execute(this);
+      }
+    }
+
+    @Override
+    public void run() {
+      if (!cancelled.get()) {
+        listener.onChange(DefaultServiceDiscovered.this);
+      }
+    }
+
+    @Override
+    public void cancel() {
+      if (cancelled.compareAndSet(false, true)) {
+        Lock writeLock = callerLock.writeLock();
+        writeLock.lock();
+        try {
+          listenerCallers.remove(this);
+        } finally {
+          writeLock.unlock();
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/b0dd8e24/twill-discovery-core/src/main/java/org/apache/twill/discovery/DiscoverableAdapter.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/main/java/org/apache/twill/discovery/DiscoverableAdapter.java b/twill-discovery-core/src/main/java/org/apache/twill/discovery/DiscoverableAdapter.java
new file mode 100644
index 0000000..0e4bcc3
--- /dev/null
+++ b/twill-discovery-core/src/main/java/org/apache/twill/discovery/DiscoverableAdapter.java
@@ -0,0 +1,102 @@
+/*
+ * 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.twill.discovery;
+
+import com.google.common.base.Charsets;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParseException;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
+
+import java.lang.reflect.Type;
+import java.net.InetSocketAddress;
+
+/**
+ * Helper class to serialize and deserialize {@link Discoverable}.
+ */
+final class DiscoverableAdapter {
+
+  private static final Gson GSON =
+    new GsonBuilder().registerTypeAdapter(Discoverable.class, new DiscoverableCodec()).create();
+
+  /**
+   * Helper function for encoding an instance of {@link Discoverable} into array of bytes.
+   * @param discoverable An instance of {@link Discoverable}
+   * @return array of bytes representing an instance of <code>discoverable</code>
+   */
+  static byte[] encode(Discoverable discoverable) {
+    return GSON.toJson(discoverable, Discoverable.class).getBytes(Charsets.UTF_8);
+  }
+
+  /**
+   * Helper function for decoding array of bytes into a {@link Discoverable} object.
+   * @param encoded representing serialized {@link Discoverable}
+   * @return {@code null} if encoded bytes are null; else an instance of {@link Discoverable}
+   */
+  static Discoverable decode(byte[] encoded) {
+    if (encoded == null) {
+      return null;
+    }
+    return GSON.fromJson(new String(encoded, Charsets.UTF_8), Discoverable.class);
+  }
+
+  private DiscoverableAdapter() {
+  }
+
+  /**
+   * SerDe for converting a {@link Discoverable} into a JSON object
+   * or from a JSON object into {@link Discoverable}.
+   */
+  private static final class DiscoverableCodec implements JsonSerializer<Discoverable>, JsonDeserializer<Discoverable> {
+
+    @Override
+    public Discoverable deserialize(JsonElement json, Type typeOfT,
+                                    JsonDeserializationContext context) throws JsonParseException {
+      JsonObject jsonObj = json.getAsJsonObject();
+      final String service = jsonObj.get("service").getAsString();
+      String hostname = jsonObj.get("hostname").getAsString();
+      int port = jsonObj.get("port").getAsInt();
+      final InetSocketAddress address = new InetSocketAddress(hostname, port);
+      return new DiscoverableWrapper(new Discoverable() {
+        @Override
+        public String getName() {
+          return service;
+        }
+
+        @Override
+        public InetSocketAddress getSocketAddress() {
+          return address;
+        }
+      });
+    }
+
+    @Override
+    public JsonElement serialize(Discoverable src, Type typeOfSrc, JsonSerializationContext context) {
+      JsonObject jsonObj = new JsonObject();
+      jsonObj.addProperty("service", src.getName());
+      jsonObj.addProperty("hostname", src.getSocketAddress().getHostName());
+      jsonObj.addProperty("port", src.getSocketAddress().getPort());
+      return jsonObj;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/b0dd8e24/twill-discovery-core/src/main/java/org/apache/twill/discovery/InMemoryDiscoveryService.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/main/java/org/apache/twill/discovery/InMemoryDiscoveryService.java b/twill-discovery-core/src/main/java/org/apache/twill/discovery/InMemoryDiscoveryService.java
index 7a9e984..2f950b8 100644
--- a/twill-discovery-core/src/main/java/org/apache/twill/discovery/InMemoryDiscoveryService.java
+++ b/twill-discovery-core/src/main/java/org/apache/twill/discovery/InMemoryDiscoveryService.java
@@ -18,11 +18,11 @@
 package org.apache.twill.discovery;
 
 import org.apache.twill.common.Cancellable;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Multimap;
+import com.google.common.collect.LinkedHashMultimap;
+import com.google.common.collect.Maps;
+import com.google.common.collect.SetMultimap;
 
-import java.util.Iterator;
+import java.util.Map;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
@@ -31,43 +31,58 @@ import java.util.concurrent.locks.ReentrantLock;
  */
 public class InMemoryDiscoveryService implements DiscoveryService, DiscoveryServiceClient {
 
-  private final Multimap<String, Discoverable> services = HashMultimap.create();
+  private final SetMultimap<String, Discoverable> services = LinkedHashMultimap.create();
+  private final Map<String, DefaultServiceDiscovered> serviceDiscoveredMap = Maps.newHashMap();
   private final Lock lock = new ReentrantLock();
 
   @Override
   public Cancellable register(final Discoverable discoverable) {
+    final Discoverable wrapper = new DiscoverableWrapper(discoverable);
+    final String serviceName = wrapper.getName();
+
     lock.lock();
     try {
-      final Discoverable wrapper = new DiscoverableWrapper(discoverable);
-      services.put(wrapper.getName(), wrapper);
-      return new Cancellable() {
-        @Override
-        public void cancel() {
-          lock.lock();
-          try {
-            services.remove(wrapper.getName(), wrapper);
-          } finally {
-            lock.unlock();
-          }
-        }
-      };
+      services.put(serviceName, wrapper);
+
+      DefaultServiceDiscovered serviceDiscovered = serviceDiscoveredMap.get(serviceName);
+      if (serviceDiscovered != null) {
+        serviceDiscovered.setDiscoverables(services.get(serviceName));
+      }
     } finally {
       lock.unlock();
     }
-  }
 
-  @Override
-  public Iterable<Discoverable> discover(final String name) {
-    return new Iterable<Discoverable>() {
+    return new Cancellable() {
       @Override
-      public Iterator<Discoverable> iterator() {
+      public void cancel() {
         lock.lock();
         try {
-          return ImmutableList.copyOf(services.get(name)).iterator();
+          services.remove(serviceName, wrapper);
+
+          DefaultServiceDiscovered serviceDiscovered = serviceDiscoveredMap.get(serviceName);
+          if (serviceDiscovered != null) {
+            serviceDiscovered.setDiscoverables(services.get(serviceName));
+          }
         } finally {
           lock.unlock();
         }
       }
     };
   }
+
+  @Override
+  public ServiceDiscovered discover(final String name) {
+    lock.lock();
+    try {
+      DefaultServiceDiscovered serviceDiscovered = serviceDiscoveredMap.get(name);
+      if (serviceDiscovered == null) {
+        serviceDiscovered = new DefaultServiceDiscovered(name);
+        serviceDiscovered.setDiscoverables(services.get(name));
+        serviceDiscoveredMap.put(name, serviceDiscovered);
+      }
+      return serviceDiscovered;
+    } finally {
+      lock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/b0dd8e24/twill-discovery-core/src/main/java/org/apache/twill/discovery/ZKDiscoveryService.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/main/java/org/apache/twill/discovery/ZKDiscoveryService.java b/twill-discovery-core/src/main/java/org/apache/twill/discovery/ZKDiscoveryService.java
index 862fcbc..0543626 100644
--- a/twill-discovery-core/src/main/java/org/apache/twill/discovery/ZKDiscoveryService.java
+++ b/twill-discovery-core/src/main/java/org/apache/twill/discovery/ZKDiscoveryService.java
@@ -25,12 +25,11 @@ import org.apache.twill.zookeeper.OperationFuture;
 import org.apache.twill.zookeeper.ZKClient;
 import org.apache.twill.zookeeper.ZKClients;
 import org.apache.twill.zookeeper.ZKOperations;
-import com.google.common.base.Charsets;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.collect.HashMultimap;
-import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
 import com.google.common.hash.Hashing;
@@ -38,14 +37,6 @@ import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
-import com.google.gson.GsonBuilder;
-import com.google.gson.JsonDeserializationContext;
-import com.google.gson.JsonDeserializer;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonObject;
-import com.google.gson.JsonParseException;
-import com.google.gson.JsonSerializationContext;
-import com.google.gson.JsonSerializer;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -54,16 +45,13 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.lang.reflect.Type;
 import java.net.InetSocketAddress;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
@@ -96,7 +84,7 @@ import java.util.concurrent.locks.ReentrantLock;
  *      &#125;);
  *      ...
  *      ...
- *      Iterable<Discoverable> services = service.discovery("service-name");
+ *      ServiceDiscovered services = service.discovery("service-name");
  *      ...
  *      }
  *    </pre>
@@ -114,7 +102,7 @@ public class ZKDiscoveryService implements DiscoveryService, DiscoveryServiceCli
   private final Multimap<Discoverable, DiscoveryCancellable> discoverables;
   private final Lock lock;
 
-  private final LoadingCache<String, Iterable<Discoverable>> services;
+  private final LoadingCache<String, ServiceDiscovered> services;
   private final ZKClient zkClient;
   private final ScheduledExecutorService retryExecutor;
 
@@ -193,7 +181,7 @@ public class ZKDiscoveryService implements DiscoveryService, DiscoveryServiceCli
   }
 
   @Override
-  public Iterable<Discoverable> discover(String service) {
+  public ServiceDiscovered discover(String service) {
     return services.getUnchecked(service);
   }
 
@@ -251,7 +239,7 @@ public class ZKDiscoveryService implements DiscoveryService, DiscoveryServiceCli
   }
 
   private OperationFuture<String> doRegister(Discoverable discoverable) {
-    byte[] discoverableBytes = encode(discoverable);
+    byte[] discoverableBytes = DiscoverableAdapter.encode(discoverable);
     return zkClient.create(getNodePath(discoverable), discoverableBytes, CreateMode.EPHEMERAL, true);
   }
 
@@ -330,14 +318,11 @@ public class ZKDiscoveryService implements DiscoveryService, DiscoveryServiceCli
   /**
    * Creates a CacheLoader for creating live Iterable for watching instances changes for a given service.
    */
-  private CacheLoader<String, Iterable<Discoverable>> createServiceLoader() {
-    return new CacheLoader<String, Iterable<Discoverable>>() {
+  private CacheLoader<String, ServiceDiscovered> createServiceLoader() {
+    return new CacheLoader<String, ServiceDiscovered>() {
       @Override
-      public Iterable<Discoverable> load(String service) throws Exception {
-        // The atomic reference is to keep the resulting Iterable live. It always contains a
-        // immutable snapshot of the latest detected set of Discoverable.
-        final AtomicReference<Iterable<Discoverable>> iterable =
-              new AtomicReference<Iterable<Discoverable>>(ImmutableList.<Discoverable>of());
+      public ServiceDiscovered load(String service) throws Exception {
+        final DefaultServiceDiscovered serviceDiscovered = new DefaultServiceDiscovered(service);
         final String serviceBase = "/" + service;
 
         // Watch for children changes in /service
@@ -356,60 +341,27 @@ public class ZKDiscoveryService implements DiscoveryService, DiscoveryServiceCli
             fetchFuture.addListener(new Runnable() {
               @Override
               public void run() {
-                ImmutableList.Builder<Discoverable> builder = ImmutableList.builder();
+                ImmutableSet.Builder<Discoverable> builder = ImmutableSet.builder();
                 for (NodeData nodeData : Futures.getUnchecked(fetchFuture)) {
                   // For successful fetch, decode the content.
                   if (nodeData != null) {
-                    Discoverable discoverable = decode(nodeData.getData());
+                    Discoverable discoverable = DiscoverableAdapter.decode(nodeData.getData());
                     if (discoverable != null) {
                       builder.add(discoverable);
                     }
                   }
                 }
-                iterable.set(builder.build());
+                serviceDiscovered.setDiscoverables(builder.build());
               }
             }, Threads.SAME_THREAD_EXECUTOR);
           }
         });
-
-        return new Iterable<Discoverable>() {
-          @Override
-          public Iterator<Discoverable> iterator() {
-            return iterable.get().iterator();
-          }
-        };
+        return serviceDiscovered;
       }
     };
   }
 
   /**
-   * Static helper function for decoding array of bytes into a {@link DiscoverableWrapper} object.
-   * @param bytes representing serialized {@link DiscoverableWrapper}
-   * @return null if bytes are null; else an instance of {@link DiscoverableWrapper}
-   */
-  private static Discoverable decode(byte[] bytes) {
-    if (bytes == null) {
-      return null;
-    }
-    String content = new String(bytes, Charsets.UTF_8);
-    return new GsonBuilder().registerTypeAdapter(Discoverable.class, new DiscoverableCodec())
-      .create()
-      .fromJson(content, Discoverable.class);
-  }
-
-  /**
-   * Static helper function for encoding an instance of {@link DiscoverableWrapper} into array of bytes.
-   * @param discoverable An instance of {@link Discoverable}
-   * @return array of bytes representing an instance of <code>discoverable</code>
-   */
-  private static byte[] encode(Discoverable discoverable) {
-    return new GsonBuilder().registerTypeAdapter(DiscoverableWrapper.class, new DiscoverableCodec())
-      .create()
-      .toJson(discoverable, DiscoverableWrapper.class)
-      .getBytes(Charsets.UTF_8);
-  }
-
-  /**
    * Inner class for cancelling (un-register) discovery service.
    */
   private final class DiscoveryCancellable implements Cancellable {
@@ -471,42 +423,5 @@ public class ZKDiscoveryService implements DiscoveryService, DiscoveryServiceCli
       LOG.debug("Service unregistered: {} {}", discoverable, path);
     }
   }
-
-  /**
-   * SerDe for converting a {@link DiscoverableWrapper} into a JSON object
-   * or from a JSON object into {@link DiscoverableWrapper}.
-   */
-  private static final class DiscoverableCodec implements JsonSerializer<Discoverable>, JsonDeserializer<Discoverable> {
-
-    @Override
-    public Discoverable deserialize(JsonElement json, Type typeOfT,
-                                    JsonDeserializationContext context) throws JsonParseException {
-      JsonObject jsonObj = json.getAsJsonObject();
-      final String service = jsonObj.get("service").getAsString();
-      String hostname = jsonObj.get("hostname").getAsString();
-      int port = jsonObj.get("port").getAsInt();
-      final InetSocketAddress address = new InetSocketAddress(hostname, port);
-      return new Discoverable() {
-        @Override
-        public String getName() {
-          return service;
-        }
-
-        @Override
-        public InetSocketAddress getSocketAddress() {
-          return address;
-        }
-      };
-    }
-
-    @Override
-    public JsonElement serialize(Discoverable src, Type typeOfSrc, JsonSerializationContext context) {
-      JsonObject jsonObj = new JsonObject();
-      jsonObj.addProperty("service", src.getName());
-      jsonObj.addProperty("hostname", src.getSocketAddress().getHostName());
-      jsonObj.addProperty("port", src.getSocketAddress().getPort());
-      return jsonObj;
-    }
-  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/b0dd8e24/twill-discovery-core/src/test/java/org/apache/twill/discovery/DiscoveryServiceTestBase.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/test/java/org/apache/twill/discovery/DiscoveryServiceTestBase.java b/twill-discovery-core/src/test/java/org/apache/twill/discovery/DiscoveryServiceTestBase.java
new file mode 100644
index 0000000..17b526b
--- /dev/null
+++ b/twill-discovery-core/src/test/java/org/apache/twill/discovery/DiscoveryServiceTestBase.java
@@ -0,0 +1,278 @@
+/*
+ * 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.twill.discovery;
+
+import org.apache.twill.common.Cancellable;
+import org.apache.twill.common.Threads;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Base class for testing different discovery service implementation.
+ */
+public abstract class DiscoveryServiceTestBase {
+
+  protected abstract Map.Entry<DiscoveryService, DiscoveryServiceClient> create();
+
+  @Test
+  public void simpleDiscoverable() throws Exception {
+    Map.Entry<DiscoveryService, DiscoveryServiceClient> entry = create();
+    DiscoveryService discoveryService = entry.getKey();
+    DiscoveryServiceClient discoveryServiceClient = entry.getValue();
+
+    // Register one service running on one host:port
+    Cancellable cancellable = register(discoveryService, "foo", "localhost", 8090);
+
+    // Discover that registered host:port.
+    ServiceDiscovered serviceDiscovered = discoveryServiceClient.discover("foo");
+    Assert.assertTrue(waitTillExpected(1, serviceDiscovered));
+
+    Discoverable discoverable = new Discoverable() {
+      @Override
+      public String getName() {
+        return "foo";
+      }
+
+      @Override
+      public InetSocketAddress getSocketAddress() {
+        return new InetSocketAddress("localhost", 8090);
+      }
+    };
+
+    // Check it exists.
+    Assert.assertTrue(serviceDiscovered.contains(discoverable));
+
+    // Remove the service
+    cancellable.cancel();
+
+    // There should be no service.
+    Assert.assertTrue(waitTillExpected(0, serviceDiscovered));
+
+    Assert.assertFalse(serviceDiscovered.contains(discoverable));
+  }
+
+  @Test
+  public void testChangeListener() throws InterruptedException {
+    Map.Entry<DiscoveryService, DiscoveryServiceClient> entry = create();
+    DiscoveryService discoveryService = entry.getKey();
+    DiscoveryServiceClient discoveryServiceClient = entry.getValue();
+
+    // Start discovery
+    String serviceName = "listener_test";
+    ServiceDiscovered serviceDiscovered = discoveryServiceClient.discover(serviceName);
+
+    // Watch for changes.
+    final BlockingQueue<List<Discoverable>> events = new ArrayBlockingQueue<List<Discoverable>>(10);
+    serviceDiscovered.watchChanges(new ServiceDiscovered.ChangeListener() {
+      @Override
+      public void onChange(ServiceDiscovered serviceDiscovered) {
+        events.add(ImmutableList.copyOf(serviceDiscovered));
+      }
+    }, Threads.SAME_THREAD_EXECUTOR);
+
+    // An empty list will be received first, as no endpoint has been registered.
+    List<Discoverable> discoverables = events.poll(5, TimeUnit.SECONDS);
+    Assert.assertNotNull(discoverables);
+    Assert.assertTrue(discoverables.isEmpty());
+
+    // Register a service
+    Cancellable cancellable = register(discoveryService, serviceName, "localhost", 10000);
+
+    discoverables = events.poll(5, TimeUnit.SECONDS);
+    Assert.assertNotNull(discoverables);
+    Assert.assertEquals(1, discoverables.size());
+
+    // Register another service endpoint
+    Cancellable cancellable2 = register(discoveryService, serviceName, "localhost", 10001);
+
+    discoverables = events.poll(5, TimeUnit.SECONDS);
+    Assert.assertNotNull(discoverables);
+    Assert.assertEquals(2, discoverables.size());
+
+    // Cancel both of them
+    cancellable.cancel();
+    cancellable2.cancel();
+
+    // There could be more than one event triggered, but the last event should be an empty list.
+    discoverables = events.poll(5, TimeUnit.SECONDS);
+    Assert.assertNotNull(discoverables);
+    if (!discoverables.isEmpty()) {
+      discoverables = events.poll(5, TimeUnit.SECONDS);
+    }
+
+    Assert.assertTrue(discoverables.isEmpty());
+  }
+
+  @Test
+  public void testCancelChangeListener() throws InterruptedException {
+    Map.Entry<DiscoveryService, DiscoveryServiceClient> entry = create();
+    DiscoveryService discoveryService = entry.getKey();
+    DiscoveryServiceClient discoveryServiceClient = entry.getValue();
+
+    String serviceName = "cancel_listener";
+    ServiceDiscovered serviceDiscovered = discoveryServiceClient.discover(serviceName);
+
+    // An executor that delay execute a Runnable. It's for testing race because listener cancel and discovery changes.
+    Executor delayExecutor = new Executor() {
+      @Override
+      public void execute(final Runnable command) {
+        Thread t = new Thread() {
+          @Override
+          public void run() {
+            try {
+              TimeUnit.SECONDS.sleep(2);
+              command.run();
+            } catch (InterruptedException e) {
+              throw Throwables.propagate(e);
+            }
+          }
+        };
+        t.start();
+      }
+    };
+
+    final BlockingQueue<List<Discoverable>> events = new ArrayBlockingQueue<List<Discoverable>>(10);
+    Cancellable cancelWatch = serviceDiscovered.watchChanges(new ServiceDiscovered.ChangeListener() {
+      @Override
+      public void onChange(ServiceDiscovered serviceDiscovered) {
+        events.add(ImmutableList.copyOf(serviceDiscovered));
+      }
+    }, delayExecutor);
+
+    // Wait for the init event call
+    Assert.assertNotNull(events.poll(3, TimeUnit.SECONDS));
+
+    // Register a new service endpoint, wait a short while and then cancel the listener
+    register(discoveryService, serviceName, "localhost", 1);
+    TimeUnit.SECONDS.sleep(1);
+    cancelWatch.cancel();
+
+    // The change listener shouldn't get any event, since the invocation is delayed by the executor.
+    Assert.assertNull(events.poll(3, TimeUnit.SECONDS));
+  }
+
+  @Test
+  public void manySameDiscoverable() throws Exception {
+    Map.Entry<DiscoveryService, DiscoveryServiceClient> entry = create();
+    DiscoveryService discoveryService = entry.getKey();
+    DiscoveryServiceClient discoveryServiceClient = entry.getValue();
+
+    List<Cancellable> cancellables = Lists.newArrayList();
+
+    cancellables.add(register(discoveryService, "manyDiscoverable", "localhost", 1));
+    cancellables.add(register(discoveryService, "manyDiscoverable", "localhost", 2));
+    cancellables.add(register(discoveryService, "manyDiscoverable", "localhost", 3));
+    cancellables.add(register(discoveryService, "manyDiscoverable", "localhost", 4));
+    cancellables.add(register(discoveryService, "manyDiscoverable", "localhost", 5));
+
+    ServiceDiscovered serviceDiscovered = discoveryServiceClient.discover("manyDiscoverable");
+    Assert.assertTrue(waitTillExpected(5, serviceDiscovered));
+
+    for (int i = 0; i < 5; i++) {
+      cancellables.get(i).cancel();
+      Assert.assertTrue(waitTillExpected(4 - i, serviceDiscovered));
+    }
+  }
+
+  @Test
+  public void multiServiceDiscoverable() throws Exception {
+    Map.Entry<DiscoveryService, DiscoveryServiceClient> entry = create();
+    DiscoveryService discoveryService = entry.getKey();
+    DiscoveryServiceClient discoveryServiceClient = entry.getValue();
+
+    List<Cancellable> cancellables = Lists.newArrayList();
+
+    cancellables.add(register(discoveryService, "service1", "localhost", 1));
+    cancellables.add(register(discoveryService, "service1", "localhost", 2));
+    cancellables.add(register(discoveryService, "service1", "localhost", 3));
+    cancellables.add(register(discoveryService, "service1", "localhost", 4));
+    cancellables.add(register(discoveryService, "service1", "localhost", 5));
+
+    cancellables.add(register(discoveryService, "service2", "localhost", 1));
+    cancellables.add(register(discoveryService, "service2", "localhost", 2));
+    cancellables.add(register(discoveryService, "service2", "localhost", 3));
+
+    cancellables.add(register(discoveryService, "service3", "localhost", 1));
+    cancellables.add(register(discoveryService, "service3", "localhost", 2));
+
+    ServiceDiscovered serviceDiscovered = discoveryServiceClient.discover("service1");
+    Assert.assertTrue(waitTillExpected(5, serviceDiscovered));
+
+    serviceDiscovered = discoveryServiceClient.discover("service2");
+    Assert.assertTrue(waitTillExpected(3, serviceDiscovered));
+
+    serviceDiscovered = discoveryServiceClient.discover("service3");
+    Assert.assertTrue(waitTillExpected(2, serviceDiscovered));
+
+    cancellables.add(register(discoveryService, "service3", "localhost", 3));
+    Assert.assertTrue(waitTillExpected(3, serviceDiscovered)); // Shows live iterator.
+
+    for (Cancellable cancellable : cancellables) {
+      cancellable.cancel();
+    }
+
+    Assert.assertTrue(waitTillExpected(0, discoveryServiceClient.discover("service1")));
+    Assert.assertTrue(waitTillExpected(0, discoveryServiceClient.discover("service2")));
+    Assert.assertTrue(waitTillExpected(0, discoveryServiceClient.discover("service3")));
+  }
+
+  protected Cancellable register(DiscoveryService service, final String name, final String host, final int port) {
+    return service.register(new Discoverable() {
+      @Override
+      public String getName() {
+        return name;
+      }
+
+      @Override
+      public InetSocketAddress getSocketAddress() {
+        return new InetSocketAddress(host, port);
+      }
+    });
+  }
+
+  protected boolean waitTillExpected(final int expected, ServiceDiscovered serviceDiscovered) {
+    final CountDownLatch latch = new CountDownLatch(1);
+    serviceDiscovered.watchChanges(new ServiceDiscovered.ChangeListener() {
+      @Override
+      public void onChange(ServiceDiscovered serviceDiscovered) {
+        if (expected == Iterables.size(serviceDiscovered)) {
+          latch.countDown();
+        }
+      }
+    }, Threads.SAME_THREAD_EXECUTOR);
+
+    try {
+      return latch.await(5, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      throw Throwables.propagate(e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/b0dd8e24/twill-discovery-core/src/test/java/org/apache/twill/discovery/InMemoryDiscoveryServiceTest.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/test/java/org/apache/twill/discovery/InMemoryDiscoveryServiceTest.java b/twill-discovery-core/src/test/java/org/apache/twill/discovery/InMemoryDiscoveryServiceTest.java
index d8cc375..d49e8a5 100644
--- a/twill-discovery-core/src/test/java/org/apache/twill/discovery/InMemoryDiscoveryServiceTest.java
+++ b/twill-discovery-core/src/test/java/org/apache/twill/discovery/InMemoryDiscoveryServiceTest.java
@@ -18,50 +18,18 @@
 
 package org.apache.twill.discovery;
 
-import org.apache.twill.common.Cancellable;
-import com.google.common.collect.Iterables;
-import org.junit.Assert;
-import org.junit.Test;
+import com.google.common.collect.Maps;
 
-import java.net.InetSocketAddress;
-import java.util.concurrent.TimeUnit;
+import java.util.Map;
 
 /**
  * Test memory based service discovery service.
  */
-public class InMemoryDiscoveryServiceTest {
-  private Cancellable register(DiscoveryService service, final String name, final String host, final int port) {
-    return service.register(new Discoverable() {
-      @Override
-      public String getName() {
-        return name;
-      }
+public class InMemoryDiscoveryServiceTest extends DiscoveryServiceTestBase {
 
-      @Override
-      public InetSocketAddress getSocketAddress() {
-        return new InetSocketAddress(host, port);
-      }
-    });
-  }
-
-  @Test
-  public void simpleDiscoverable() throws Exception {
+  @Override
+  protected Map.Entry<DiscoveryService, DiscoveryServiceClient> create() {
     DiscoveryService discoveryService = new InMemoryDiscoveryService();
-    DiscoveryServiceClient discoveryServiceClient = (DiscoveryServiceClient) discoveryService;
-
-    // Register one service running on one host:port
-    Cancellable cancellable = register(discoveryService, "foo", "localhost", 8090);
-    Iterable<Discoverable> discoverables = discoveryServiceClient.discover("foo");
-
-    // Discover that registered host:port.
-    Assert.assertTrue(Iterables.size(discoverables) == 1);
-
-    // Remove the service
-    cancellable.cancel();
-
-    // There should be no service.
-    discoverables = discoveryServiceClient.discover("foo");
-    TimeUnit.MILLISECONDS.sleep(100);
-    Assert.assertTrue(Iterables.size(discoverables) == 0);
+    return Maps.immutableEntry(discoveryService, (DiscoveryServiceClient) discoveryService);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/b0dd8e24/twill-discovery-core/src/test/java/org/apache/twill/discovery/ZKDiscoveryServiceTest.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/test/java/org/apache/twill/discovery/ZKDiscoveryServiceTest.java b/twill-discovery-core/src/test/java/org/apache/twill/discovery/ZKDiscoveryServiceTest.java
index feee8db..6f0cde0 100644
--- a/twill-discovery-core/src/test/java/org/apache/twill/discovery/ZKDiscoveryServiceTest.java
+++ b/twill-discovery-core/src/test/java/org/apache/twill/discovery/ZKDiscoveryServiceTest.java
@@ -25,8 +25,7 @@ import org.apache.twill.zookeeper.RetryStrategies;
 import org.apache.twill.zookeeper.ZKClientService;
 import org.apache.twill.zookeeper.ZKClientServices;
 import org.apache.twill.zookeeper.ZKClients;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.Futures;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -35,14 +34,13 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.InetSocketAddress;
-import java.util.List;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 /**
  * Test Zookeeper based discovery service.
  */
-public class ZKDiscoveryServiceTest {
+public class ZKDiscoveryServiceTest extends DiscoveryServiceTestBase {
   private static final Logger LOG = LoggerFactory.getLogger(ZKDiscoveryServiceTest.class);
 
   private static InMemoryZKServer zkServer;
@@ -66,41 +64,17 @@ public class ZKDiscoveryServiceTest {
     Futures.getUnchecked(Services.chainStop(zkClient, zkServer));
   }
 
-  private Cancellable register(DiscoveryService service, final String name, final String host, final int port) {
-    return service.register(new Discoverable() {
-      @Override
-      public String getName() {
-        return name;
-      }
-
-      @Override
-      public InetSocketAddress getSocketAddress() {
-        return new InetSocketAddress(host, port);
-      }
-    });
-  }
-
-
-  private boolean waitTillExpected(int expected, Iterable<Discoverable> discoverables) throws Exception {
-    for (int i = 0; i < 10; ++i) {
-      TimeUnit.MILLISECONDS.sleep(10);
-      if (Iterables.size(discoverables) == expected) {
-        return true;
-      }
-    }
-    return (Iterables.size(discoverables) == expected);
-  }
-
   @Test (timeout = 5000)
   public void testDoubleRegister() throws Exception {
-    ZKDiscoveryService discoveryService = new ZKDiscoveryService(zkClient);
-    DiscoveryServiceClient discoveryServiceClient = discoveryService;
+    Map.Entry<DiscoveryService, DiscoveryServiceClient> entry = create();
+    DiscoveryService discoveryService = entry.getKey();
+    DiscoveryServiceClient discoveryServiceClient = entry.getValue();
 
     // Register on the same host port, it shouldn't fail.
     Cancellable cancellable = register(discoveryService, "test_double_reg", "localhost", 54321);
     Cancellable cancellable2 = register(discoveryService, "test_double_reg", "localhost", 54321);
 
-    Iterable<Discoverable> discoverables = discoveryServiceClient.discover("test_double_reg");
+    ServiceDiscovered discoverables = discoveryServiceClient.discover("test_double_reg");
 
     Assert.assertTrue(waitTillExpected(1, discoverables));
 
@@ -116,7 +90,7 @@ public class ZKDiscoveryServiceTest {
     zkClient2.startAndWait();
 
     try {
-      ZKDiscoveryService discoveryService2 = new ZKDiscoveryService(zkClient2);
+      DiscoveryService discoveryService2 = new ZKDiscoveryService(zkClient2);
       cancellable2 = register(discoveryService2, "test_multi_client", "localhost", 54321);
 
       // Schedule a thread to shutdown zkClient2.
@@ -143,12 +117,13 @@ public class ZKDiscoveryServiceTest {
 
   @Test
   public void testSessionExpires() throws Exception {
-    ZKDiscoveryService discoveryService = new ZKDiscoveryService(zkClient);
-    DiscoveryServiceClient discoveryServiceClient = discoveryService;
+    Map.Entry<DiscoveryService, DiscoveryServiceClient> entry = create();
+    DiscoveryService discoveryService = entry.getKey();
+    DiscoveryServiceClient discoveryServiceClient = entry.getValue();
 
     Cancellable cancellable = register(discoveryService, "test_expires", "localhost", 54321);
 
-    Iterable<Discoverable> discoverables = discoveryServiceClient.discover("test_expires");
+    ServiceDiscovered discoverables = discoveryServiceClient.discover("test_expires");
 
     // Discover that registered host:port.
     Assert.assertTrue(waitTillExpected(1, discoverables));
@@ -168,86 +143,11 @@ public class ZKDiscoveryServiceTest {
     Assert.assertTrue(waitTillExpected(0, discoverables));
   }
 
-  @Test
-  public void simpleDiscoverable() throws Exception {
-    DiscoveryService discoveryService = new ZKDiscoveryService(zkClient);
-    DiscoveryServiceClient discoveryServiceClient = new ZKDiscoveryService(zkClient);
-
-    // Register one service running on one host:port
-    Cancellable cancellable = register(discoveryService, "foo", "localhost", 8090);
-    Iterable<Discoverable> discoverables = discoveryServiceClient.discover("foo");
-
-    // Discover that registered host:port.
-    Assert.assertTrue(waitTillExpected(1, discoverables));
-
-    // Remove the service
-    cancellable.cancel();
-
-    // There should be no service.
-
-    discoverables = discoveryServiceClient.discover("foo");
-
-    Assert.assertTrue(waitTillExpected(0, discoverables));
-  }
-
-  @Test
-  public void manySameDiscoverable() throws Exception {
-    List<Cancellable> cancellables = Lists.newArrayList();
+  @Override
+  protected Map.Entry<DiscoveryService, DiscoveryServiceClient> create() {
     DiscoveryService discoveryService = new ZKDiscoveryService(zkClient);
     DiscoveryServiceClient discoveryServiceClient = new ZKDiscoveryService(zkClient);
 
-    cancellables.add(register(discoveryService, "manyDiscoverable", "localhost", 1));
-    cancellables.add(register(discoveryService, "manyDiscoverable", "localhost", 2));
-    cancellables.add(register(discoveryService, "manyDiscoverable", "localhost", 3));
-    cancellables.add(register(discoveryService, "manyDiscoverable", "localhost", 4));
-    cancellables.add(register(discoveryService, "manyDiscoverable", "localhost", 5));
-
-    Iterable<Discoverable> discoverables = discoveryServiceClient.discover("manyDiscoverable");
-    Assert.assertTrue(waitTillExpected(5, discoverables));
-
-    for (int i = 0; i < 5; i++) {
-      cancellables.get(i).cancel();
-      Assert.assertTrue(waitTillExpected(4 - i, discoverables));
-    }
-  }
-
-  @Test
-  public void multiServiceDiscoverable() throws Exception {
-    List<Cancellable> cancellables = Lists.newArrayList();
-    DiscoveryService discoveryService = new ZKDiscoveryService(zkClient);
-    DiscoveryServiceClient discoveryServiceClient = new ZKDiscoveryService(zkClient);
-
-    cancellables.add(register(discoveryService, "service1", "localhost", 1));
-    cancellables.add(register(discoveryService, "service1", "localhost", 2));
-    cancellables.add(register(discoveryService, "service1", "localhost", 3));
-    cancellables.add(register(discoveryService, "service1", "localhost", 4));
-    cancellables.add(register(discoveryService, "service1", "localhost", 5));
-
-    cancellables.add(register(discoveryService, "service2", "localhost", 1));
-    cancellables.add(register(discoveryService, "service2", "localhost", 2));
-    cancellables.add(register(discoveryService, "service2", "localhost", 3));
-
-    cancellables.add(register(discoveryService, "service3", "localhost", 1));
-    cancellables.add(register(discoveryService, "service3", "localhost", 2));
-
-    Iterable<Discoverable> discoverables = discoveryServiceClient.discover("service1");
-    Assert.assertTrue(waitTillExpected(5, discoverables));
-
-    discoverables = discoveryServiceClient.discover("service2");
-    Assert.assertTrue(waitTillExpected(3, discoverables));
-
-    discoverables = discoveryServiceClient.discover("service3");
-    Assert.assertTrue(waitTillExpected(2, discoverables));
-
-    cancellables.add(register(discoveryService, "service3", "localhost", 3));
-    Assert.assertTrue(waitTillExpected(3, discoverables)); // Shows live iterator.
-
-    for (Cancellable cancellable : cancellables) {
-      cancellable.cancel();
-    }
-
-    Assert.assertTrue(waitTillExpected(0, discoveryServiceClient.discover("service1")));
-    Assert.assertTrue(waitTillExpected(0, discoveryServiceClient.discover("service2")));
-    Assert.assertTrue(waitTillExpected(0, discoveryServiceClient.discover("service3")));
+    return Maps.immutableEntry(discoveryService, discoveryServiceClient);
   }
 }


[18/50] [abbrv] git commit: Bump version to 0.1.0-incubating to prepare for releasing.

Posted by ch...@apache.org.
Bump version to 0.1.0-incubating to prepare for releasing.

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

Branch: refs/heads/site
Commit: df8e2786d848fb94037e89cc44b31405d1f9fb14
Parents: 78eb1e1
Author: Terence Yim <te...@continuuity.com>
Authored: Tue Jan 28 14:09:49 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Tue Jan 28 14:09:49 2014 -0800

----------------------------------------------------------------------
 pom.xml                      | 2 +-
 twill-api/pom.xml            | 2 +-
 twill-common/pom.xml         | 2 +-
 twill-core/pom.xml           | 2 +-
 twill-discovery-api/pom.xml  | 2 +-
 twill-discovery-core/pom.xml | 2 +-
 twill-yarn/pom.xml           | 2 +-
 twill-zookeeper/pom.xml      | 2 +-
 8 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/df8e2786/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f960b75..df87929 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
 
     <groupId>org.apache.twill</groupId>
     <artifactId>twill-parent</artifactId>
-    <version>0.0.1-incubating-SNAPSHOT</version>
+    <version>0.1.0-incubating-SNAPSHOT</version>
     <packaging>pom</packaging>
     <name>Apache Twill</name>
     <url>http://twill.incubator.apache.org</url>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/df8e2786/twill-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-api/pom.xml b/twill-api/pom.xml
index 29288b4..a018adf 100644
--- a/twill-api/pom.xml
+++ b/twill-api/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.twill</groupId>
         <artifactId>twill-parent</artifactId>
-        <version>0.0.1-incubating-SNAPSHOT</version>
+        <version>0.1.0-incubating-SNAPSHOT</version>
     </parent>
 
     <artifactId>twill-api</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/df8e2786/twill-common/pom.xml
----------------------------------------------------------------------
diff --git a/twill-common/pom.xml b/twill-common/pom.xml
index 1d86ed1..c39c87a 100644
--- a/twill-common/pom.xml
+++ b/twill-common/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.0.1-incubating-SNAPSHOT</version>
+        <version>0.1.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/df8e2786/twill-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-core/pom.xml b/twill-core/pom.xml
index ae38146..6bf4e0f 100644
--- a/twill-core/pom.xml
+++ b/twill-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.0.1-incubating-SNAPSHOT</version>
+        <version>0.1.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/df8e2786/twill-discovery-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-api/pom.xml b/twill-discovery-api/pom.xml
index 296bdde..78b5d26 100644
--- a/twill-discovery-api/pom.xml
+++ b/twill-discovery-api/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.0.1-incubating-SNAPSHOT</version>
+        <version>0.1.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/df8e2786/twill-discovery-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-core/pom.xml b/twill-discovery-core/pom.xml
index 4f89bfe..4531fbb 100644
--- a/twill-discovery-core/pom.xml
+++ b/twill-discovery-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.0.1-incubating-SNAPSHOT</version>
+        <version>0.1.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/df8e2786/twill-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-yarn/pom.xml b/twill-yarn/pom.xml
index 6a63db7..b1493e8 100644
--- a/twill-yarn/pom.xml
+++ b/twill-yarn/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.0.1-incubating-SNAPSHOT</version>
+        <version>0.1.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/df8e2786/twill-zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/twill-zookeeper/pom.xml b/twill-zookeeper/pom.xml
index 6c917d5..a9d7324 100644
--- a/twill-zookeeper/pom.xml
+++ b/twill-zookeeper/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.0.1-incubating-SNAPSHOT</version>
+        <version>0.1.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 


[15/50] [abbrv] git commit: Added YARN-2.0.x compatible module into source jar.

Posted by ch...@apache.org.
Added YARN-2.0.x compatible module into source jar.

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

Branch: refs/heads/site
Commit: a28f3c01ce5bdb459fd4e4339671181d8985452d
Parents: bbdb212
Author: Terence Yim <ch...@apache.org>
Authored: Tue Jan 28 01:29:41 2014 -0800
Committer: Terence Yim <ch...@apache.org>
Committed: Tue Jan 28 01:29:41 2014 -0800

----------------------------------------------------------------------
 pom.xml | 24 ++++++++++++++++++++++++
 1 file changed, 24 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/a28f3c01/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a95ea6f..01c0648 100644
--- a/pom.xml
+++ b/pom.xml
@@ -427,6 +427,18 @@
                                     </sources>
                                 </configuration>
                             </execution>
+                            <execution>
+                                <id>add-source-2.0</id>
+                                <phase>prepare-package</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>src/main/hadoop20</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
                         </executions>
                     </plugin>
                 </plugins>
@@ -459,6 +471,18 @@
                                     </sources>
                                 </configuration>
                             </execution>
+                            <execution>
+                                <id>add-source-2.0</id>
+                                <phase>prepare-package</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>src/main/hadoop20</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
                         </executions>
                     </plugin>
                 </plugins>


[12/50] [abbrv] git commit: [TWILL-35] Enabling checkstyle.

Posted by ch...@apache.org.
[TWILL-35] Enabling checkstyle.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 098a0cac2c72c0d0f9627477d1ae6b6d7668291c
Parents: 61a602a
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Jan 27 16:00:45 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Jan 27 17:07:53 2014 -0800

----------------------------------------------------------------------
 checkstyle.xml                                  | 356 +++++++++++++++++++
 pom.xml                                         |  26 ++
 .../java/org/apache/twill/api/LocalFile.java    |   2 +-
 .../apache/twill/api/ResourceSpecification.java |   3 +-
 .../org/apache/twill/api/TwillController.java   |   2 +-
 .../twill/api/TwillRunnableSpecification.java   |   2 +-
 .../apache/twill/api/TwillSpecification.java    |  10 +-
 .../DefaultEventHandlerSpecification.java       |   5 +-
 .../apache/twill/internal/DefaultLocalFile.java |   2 +-
 .../twill/internal/DefaultResourceReport.java   |   4 +-
 .../internal/DefaultRuntimeSpecification.java   |   2 +-
 .../DefaultTwillRunnableSpecification.java      |   2 +-
 .../internal/DefaultTwillSpecification.java     |   8 +-
 .../java/org/apache/twill/internal/RunIds.java  |   4 +-
 .../org/apache/twill/filesystem/Location.java   |   2 +-
 .../AbstractExecutionServiceController.java     |  11 +-
 .../twill/internal/AbstractTwillController.java |   2 +-
 .../internal/AbstractZKServiceController.java   |  12 +-
 .../java/org/apache/twill/internal/Configs.java |   8 +-
 .../org/apache/twill/internal/Constants.java    |   2 +-
 .../internal/SingleRunnableApplication.java     |   2 +-
 .../internal/TwillContainerController.java      |   2 +-
 .../twill/internal/TwillContainerLauncher.java  |   2 +-
 .../org/apache/twill/internal/ZKMessages.java   |   8 +-
 .../twill/internal/ZKServiceDecorator.java      |  32 +-
 .../twill/internal/json/ArgumentsCodec.java     |  10 +-
 .../twill/internal/json/LocalFileCodec.java     |   4 +-
 .../internal/json/ResourceReportAdapter.java    |   4 +-
 .../internal/json/ResourceReportCodec.java      |  12 +-
 .../json/ResourceSpecificationCodec.java        |   4 +-
 .../json/RuntimeSpecificationCodec.java         |  16 +-
 .../twill/internal/json/StateNodeCodec.java     |   9 +-
 .../internal/json/TwillRunResourcesCodec.java   |   4 +-
 .../json/TwillRunnableSpecificationCodec.java   |  10 +-
 .../internal/json/TwillSpecificationCodec.java  |  24 +-
 .../internal/kafka/EmbeddedKafkaServer.java     |   1 -
 .../twill/internal/logging/LogEntryDecoder.java |   4 +-
 .../twill/internal/state/MessageCodec.java      |   6 +-
 .../twill/internal/state/SimpleMessage.java     |   2 +-
 .../apache/twill/internal/state/StateNode.java  |   2 +-
 .../twill/internal/state/SystemMessages.java    |   2 +-
 .../twill/kafka/client/KafkaPublisher.java      |   2 +-
 .../apache/twill/internal/ControllerTest.java   |  10 +-
 .../twill/internal/state/MessageCodecTest.java  |   2 +-
 .../internal/state/ZKServiceDecoratorTest.java  |  14 +-
 .../internal/utils/ApplicationBundlerTest.java  |   6 +-
 .../discovery/DefaultServiceDiscovered.java     |   2 +-
 .../discovery/InMemoryDiscoveryService.java     |   2 +-
 .../twill/discovery/ZKDiscoveryService.java     |  22 +-
 .../discovery/DiscoveryServiceTestBase.java     |   6 +-
 .../twill/discovery/ZKDiscoveryServiceTest.java |   4 +-
 .../internal/yarn/Hadoop20YarnAMClient.java     |  10 +-
 .../internal/yarn/Hadoop20YarnAppClient.java    |  10 +-
 .../internal/yarn/Hadoop20YarnNMClient.java     |   2 +-
 .../internal/yarn/ports/AllocationResponse.java |   2 +-
 .../yarn/ports/AllocationResponses.java         |   6 +-
 .../internal/yarn/Hadoop21YarnAMClient.java     |   4 +-
 .../internal/yarn/Hadoop21YarnAppClient.java    |  10 +-
 .../internal/yarn/Hadoop21YarnNMClient.java     |   2 +-
 .../twill/internal/AbstractTwillService.java    |   6 +-
 .../org/apache/twill/internal/ServiceMain.java  |  12 +-
 .../appmaster/ApplicationMasterMain.java        |   8 +-
 .../ApplicationMasterProcessLauncher.java       |   8 +-
 .../appmaster/ApplicationMasterService.java     |   2 +-
 .../appmaster/ApplicationSubmitter.java         |   2 +-
 .../appmaster/RunnableContainerRequest.java     |   4 +-
 .../appmaster/RunnableProcessLauncher.java      |   4 +-
 .../internal/appmaster/RunningContainers.java   |  28 +-
 .../internal/appmaster/TrackerService.java      |   4 +-
 .../internal/container/TwillContainerMain.java  |  18 +-
 .../container/TwillContainerService.java        |  19 +-
 .../yarn/AbstractYarnProcessLauncher.java       |   8 +-
 .../twill/internal/yarn/YarnAMClient.java       |   2 +-
 .../twill/internal/yarn/YarnAppClient.java      |   4 +-
 .../internal/yarn/YarnApplicationReport.java    |   4 +-
 .../internal/yarn/YarnContainerStatus.java      |   2 +-
 .../twill/internal/yarn/YarnLaunchContext.java  |   2 +-
 .../twill/internal/yarn/YarnNMClient.java       |   2 +-
 .../apache/twill/internal/yarn/YarnUtils.java   |  10 +-
 .../twill/yarn/LocationSecureStoreUpdater.java  |   6 +-
 .../apache/twill/yarn/ResourceReportClient.java |   4 +-
 .../org/apache/twill/yarn/YarnSecureStore.java  |   2 +-
 .../apache/twill/yarn/YarnTwillController.java  |  14 +-
 .../twill/yarn/YarnTwillRunnerService.java      |  60 ++--
 .../org/apache/twill/yarn/BaseYarnTest.java     |   4 +-
 .../twill/yarn/DistributeShellTestRun.java      |  16 +-
 .../org/apache/twill/yarn/DistributedShell.java |  10 +-
 .../apache/twill/yarn/EchoServerTestRun.java    |  32 +-
 .../twill/yarn/FailureRestartTestRun.java       |   3 +
 .../org/apache/twill/yarn/LocalFileTestRun.java |  41 ++-
 .../twill/yarn/ProvisionTimeoutTestRun.java     |  24 +-
 .../org/apache/twill/yarn/SocketServer.java     |   2 +-
 .../apache/twill/yarn/TaskCompletedTestRun.java |   3 +
 .../internal/zookeeper/BasicNodeChildren.java   |   2 +-
 .../twill/internal/zookeeper/BasicNodeData.java |   2 +-
 .../zookeeper/DefaultZKClientService.java       |  12 +-
 .../zookeeper/FailureRetryZKClient.java         |   6 +-
 .../internal/zookeeper/NamespaceZKClient.java   |   4 +-
 .../zookeeper/RewatchOnExpireWatcher.java       |   4 +-
 .../zookeeper/RewatchOnExpireZKClient.java      |   4 +-
 .../zookeeper/SettableOperationFuture.java      |   6 +-
 .../apache/twill/zookeeper/ZKClientService.java |   2 +-
 .../apache/twill/zookeeper/ZKOperations.java    |   6 +-
 .../apache/twill/zookeeper/ZKClientTest.java    |   4 +-
 104 files changed, 779 insertions(+), 384 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/checkstyle.xml
----------------------------------------------------------------------
diff --git a/checkstyle.xml b/checkstyle.xml
new file mode 100644
index 0000000..5fcce66
--- /dev/null
+++ b/checkstyle.xml
@@ -0,0 +1,356 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!DOCTYPE module PUBLIC
+    "-//Puppy Crawl//DTD Check Configuration 1.3//EN"
+    "http://www.puppycrawl.com/dtds/configuration_1_3.dtd">
+
+<!-- This is a checkstyle configuration file. For descriptions of
+what the following rules do, please see the checkstyle configuration
+page at http://checkstyle.sourceforge.net/config.html -->
+
+<module name="Checker">
+
+  <module name="FileTabCharacter">
+    <!-- Checks that there are no tab characters in the file.
+    -->
+  </module>
+
+  <!--
+
+  LENGTH CHECKS FOR FILES
+
+  -->
+
+  <module name="FileLength">
+    <property name="max" value="3000"/>
+    <property name="severity" value="warning"/>
+  </module>
+
+
+  <module name="NewlineAtEndOfFile">
+    <property name="lineSeparator" value="lf"/>
+  </module>
+
+  <module name="RegexpSingleline">
+    <!-- Checks that FIXME is not used in comments.  TODO is preferred.
+    -->
+    <property name="format" value="((//.*)|(\*.*))FIXME" />
+    <property name="message" value='TODO is preferred to FIXME.  e.g. "TODO: (ENG-123) -  Refactor when v2 is released."' />
+  </module>
+
+  <module name="RegexpSingleline">
+    <!-- Checks that TODOs are named with some basic formatting. Checks for the following pattern  TODO: ( 
+    -->
+    <property name="format" value="((//.*)|(\*.*))TODO[^: (]" />
+    <property name="message" value='All TODOs should be named.  e.g. "TODO: (ENG-123) - Refactor when v2 is released."' />
+  </module>
+
+  <!--<module name="JavadocPackage">-->
+    <!--&lt;!&ndash; Checks that each Java package has a Javadoc file used for commenting.-->
+      <!--Only allows a package-info.java, not package.html. &ndash;&gt;-->
+    <!--<property name="severity" value="warning"/>-->
+  <!--</module>-->
+
+  <!-- All Java AST specific tests live under TreeWalker module. -->
+  <module name="TreeWalker">
+
+    <!--
+
+    IMPORT CHECKS
+
+    -->
+
+    <module name="AvoidStarImport">
+      <property name="allowClassImports" value="false"/>
+      <property name="allowStaticMemberImports" value="false"/>
+    </module>
+
+    <module name="AvoidStaticImport">
+    </module>
+
+    <module name="RedundantImport">
+      <!-- Checks for redundant import statements. -->
+      <property name="severity" value="error"/>
+    </module>
+
+    <module name="ImportOrder">
+      <!-- Checks for out of order import statements. -->
+      <property name="severity" value="warning"/>
+      <property name="groups" value="ch,com,junit,kafka,net,org,/^javax?\./"/>
+      <!-- This ensures that static imports go to the end. -->
+      <property name="option" value="bottom"/>
+      <property name="tokens" value="STATIC_IMPORT, IMPORT"/>
+    </module>
+
+    <!--
+
+    METHOD LENGTH CHECKS
+
+    -->
+
+    <module name="MethodLength">
+      <property name="tokens" value="METHOD_DEF"/>
+      <property name="max" value="300"/>
+      <property name="countEmpty" value="false"/>
+      <property name="severity" value="warning"/>
+   </module>
+   
+    <!--
+
+    JAVADOC CHECKS
+
+    -->
+
+    <!-- Checks for Javadoc comments.                     -->
+    <!-- See http://checkstyle.sf.net/config_javadoc.html -->
+    <module name="JavadocMethod">
+      <property name="scope" value="protected"/>
+      <property name="severity" value="warning"/>
+      <property name="allowMissingJavadoc" value="true"/>
+      <property name="allowMissingParamTags" value="true"/>
+      <property name="allowMissingReturnTag" value="true"/>
+      <property name="allowMissingThrowsTags" value="true"/>
+      <property name="allowThrowsTagsForSubclasses" value="true"/>
+      <property name="allowUndeclaredRTE" value="true"/>
+    </module>
+
+    <module name="JavadocType">
+      <property name="scope" value="protected"/>
+      <property name="severity" value="error"/>
+    </module>
+
+    <module name="JavadocStyle">
+      <property name="severity" value="warning"/>
+    </module>
+
+    <!--
+
+    NAMING CHECKS
+
+    -->
+
+    <!-- Item 38 - Adhere to generally accepted naming conventions -->
+
+    <module name="PackageName">
+      <!-- Validates identifiers for package names against the
+        supplied expression. -->
+      <!-- Here the default checkstyle rule restricts package name parts to
+        seven characters, this is not in line with common practice at Google.
+      -->
+      <property name="format" value="^[a-z]+(\.[a-z][a-z0-9]{1,})*$"/>
+      <property name="severity" value="warning"/>
+    </module>
+
+    <module name="TypeNameCheck">
+      <!-- Validates static, final fields against the
+      expression "^[A-Z][a-zA-Z0-9]*$". -->
+      <metadata name="altname" value="TypeName"/>
+      <property name="severity" value="warning"/>
+    </module>
+
+    <module name="ConstantNameCheck">
+      <!-- Validates non-private, static, final fields against the supplied
+      public/package final fields "^[A-Z][A-Z0-9]*(_[A-Z0-9]+)*$". -->
+      <metadata name="altname" value="ConstantName"/>
+      <property name="applyToPublic" value="true"/>
+      <property name="applyToProtected" value="true"/>
+      <property name="applyToPackage" value="true"/>
+      <property name="applyToPrivate" value="false"/>
+      <property name="format" value="^([A-Z][A-Z0-9]*(_[A-Z0-9]+)*|FLAG_.*)$"/>
+      <message key="name.invalidPattern"
+               value="Variable ''{0}'' should be in ALL_CAPS (if it is a constant) or be private (otherwise)."/>
+      <property name="severity" value="warning"/>
+    </module>
+
+    <module name="StaticVariableNameCheck">
+      <!-- Validates static, non-final fields against the supplied
+      expression "^[a-z][a-zA-Z0-9]*_?$". -->
+      <metadata name="altname" value="StaticVariableName"/>
+      <property name="applyToPublic" value="true"/>
+      <property name="applyToProtected" value="true"/>
+      <property name="applyToPackage" value="true"/>
+      <property name="applyToPrivate" value="true"/>
+      <property name="format" value="^[a-z][a-zA-Z0-9]*_?$"/>
+      <property name="severity" value="warning"/>
+    </module>
+
+    <module name="MemberNameCheck">
+      <!-- Validates non-static members against the supplied expression. -->
+      <metadata name="altname" value="MemberName"/>
+      <property name="applyToPublic" value="true"/>
+      <property name="applyToProtected" value="true"/>
+      <property name="applyToPackage" value="true"/>
+      <property name="applyToPrivate" value="true"/>
+      <property name="format" value="^[a-z][a-zA-Z0-9]*$"/>
+      <property name="severity" value="warning"/>
+    </module>
+
+    <module name="MethodNameCheck">
+      <!-- Validates identifiers for method names. -->
+      <metadata name="altname" value="MethodName"/>
+      <property name="format" value="^[a-z][a-zA-Z0-9]*(_[a-zA-Z0-9]+)*$"/>
+      <property name="severity" value="warning"/>
+    </module>
+
+    <module name="ParameterName">
+      <!-- Validates identifiers for method parameters against the
+        expression "^[a-z][a-zA-Z0-9]*$". -->
+      <property name="severity" value="warning"/>
+    </module>
+
+    <module name="LocalFinalVariableName">
+      <!-- Validates identifiers for local final variables against the
+        expression "^[a-z][a-zA-Z0-9]*$". -->
+      <property name="severity" value="warning"/>
+    </module>
+
+    <module name="LocalVariableName">
+      <!-- Validates identifiers for local variables against the
+        expression "^[a-z][a-zA-Z0-9]*$". -->
+      <property name="severity" value="warning"/>
+    </module>
+
+
+    <!--
+
+    LENGTH and CODING CHECKS
+
+    -->
+
+    <module name="LineLength">
+      <!-- Checks if a line is too long. -->
+      <property name="max" value="120" default="120"/>
+      <property name="severity" value="error"/>
+
+      <!--
+        The default ignore pattern exempts the following elements:
+          - import statements
+          - long URLs inside comments
+      -->
+
+      <property name="ignorePattern"
+          value="${com.puppycrawl.tools.checkstyle.checks.sizes.LineLength.ignorePattern}"
+          default="^(package .*;\s*)|(import .*;\s*)|( *\* *https?://.*)$"/>
+    </module>
+
+    <module name="LeftCurly">
+      <!-- Checks for placement of the left curly brace ('{'). -->
+      <property name="severity" value="error"/>
+    </module>
+
+    <module name="RightCurly">
+      <!-- Checks right curlies on CATCH, ELSE, and TRY blocks are on
+      the same line. e.g., the following example is fine:
+      <pre>
+        if {
+          ...
+        } else
+      </pre>
+      -->
+      <!-- This next example is not fine:
+      <pre>
+        if {
+          ...
+        }
+        else
+      </pre>
+      -->
+      <property name="option" value="same"/>
+      <property name="severity" value="error"/>
+    </module>
+
+    <!-- Checks for braces around if and else blocks -->
+    <module name="NeedBraces">
+      <property name="severity" value="error"/>
+      <property name="tokens" value="LITERAL_IF, LITERAL_ELSE, LITERAL_FOR, LITERAL_WHILE, LITERAL_DO"/>
+    </module>
+
+    <module name="UpperEll">
+      <!-- Checks that long constants are defined with an upper ell.-->
+      <property name="severity" value="error"/>
+    </module>
+
+    <module name="FallThrough">
+      <!-- Warn about falling through to the next case statement.  Similar to
+      javac -Xlint:fallthrough, but the check is suppressed if a single-line comment
+      on the last non-blank line preceding the fallen-into case contains 'fall through' (or
+      some other variants which we don't publicized to promote consistency).
+      -->
+      <property name="reliefPattern"
+       value="fall through|Fall through|fallthru|Fallthru|falls through|Falls through|fallthrough|Fallthrough|No break|NO break|no break|continue on"/>
+      <property name="severity" value="error"/>
+    </module>
+
+
+    <!--
+
+    MODIFIERS CHECKS
+
+    -->
+
+    <module name="ModifierOrder">
+      <!-- Warn if modifier order is inconsistent with JLS3 8.1.1, 8.3.1, and
+           8.4.3.  The prescribed order is:
+           public, protected, private, abstract, static, final, transient, volatile,
+           synchronized, native, strictfp
+        -->
+    </module>
+
+
+    <!--
+
+    WHITESPACE CHECKS
+
+    -->
+
+    <module name="WhitespaceAround">
+      <!-- Checks that various tokens are surrounded by whitespace.
+           This includes most binary operators and keywords followed
+           by regular or curly braces.
+      -->
+      <property name="tokens" value="ASSIGN, BAND, BAND_ASSIGN, BOR,
+        BOR_ASSIGN, BSR, BSR_ASSIGN, BXOR, BXOR_ASSIGN, COLON, DIV, DIV_ASSIGN,
+        EQUAL, GE, GT, LAND, LCURLY, LE, LITERAL_CATCH, LITERAL_DO, LITERAL_ELSE,
+        LITERAL_FINALLY, LITERAL_FOR, LITERAL_IF, LITERAL_RETURN,
+        LITERAL_SYNCHRONIZED, LITERAL_TRY, LITERAL_WHILE, LOR, LT, MINUS,
+        MINUS_ASSIGN, MOD, MOD_ASSIGN, NOT_EQUAL, PLUS, PLUS_ASSIGN, QUESTION,
+        SL, SL_ASSIGN, SR_ASSIGN, STAR, STAR_ASSIGN"/>
+      <property name="allowEmptyConstructors" value="true"/>
+      <property name="allowEmptyMethods" value="true"/>
+      <property name="severity" value="error"/>
+    </module>
+
+    <module name="WhitespaceAfter">
+      <!-- Checks that commas, semicolons and typecasts are followed by
+           whitespace.
+      -->
+      <property name="tokens" value="COMMA, SEMI, TYPECAST"/>
+    </module>
+
+    <module name="NoWhitespaceAfter">
+      <!-- Checks that there is no whitespace after various unary operators.
+           Linebreaks are allowed.
+      -->
+      <property name="tokens" value="BNOT, DEC, DOT, INC, LNOT, UNARY_MINUS,
+        UNARY_PLUS"/>
+      <property name="allowLineBreaks" value="true"/>
+      <property name="severity" value="error"/>
+    </module>
+
+    <module name="NoWhitespaceBefore">
+      <!-- Checks that there is no whitespace before various unary operators.
+           Linebreaks are allowed.
+      -->
+      <property name="tokens" value="SEMI, DOT, POST_DEC, POST_INC"/>
+      <property name="allowLineBreaks" value="true"/>
+      <property name="severity" value="error"/>
+    </module>
+
+    <module name="ParenPad">
+      <!-- Checks that there is no whitespace before close parens or after
+           open parens.
+      -->
+      <property name="severity" value="error"/>
+    </module>
+
+  </module>
+</module>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 217a0ce..dbebdae 100644
--- a/pom.xml
+++ b/pom.xml
@@ -235,6 +235,28 @@
                         </execution>
                     </executions>
                 </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-checkstyle-plugin</artifactId>
+                    <version>2.11</version>
+                    <executions>
+                        <execution>
+                            <id>validate</id>
+                            <phase>validate</phase>
+                            <configuration>
+                                <configLocation>checkstyle.xml</configLocation>
+                                <encoding>UTF-8</encoding>
+                                <consoleOutput>true</consoleOutput>
+                                <failsOnError>true</failsOnError>
+                                <includeTestSourceDirectory>true</includeTestSourceDirectory>
+                                <sourceDirectory>${project.build.sourceDirectory}/..</sourceDirectory>
+                            </configuration>
+                            <goals>
+                                <goal>check</goal>
+                            </goals>
+                        </execution>
+                    </executions>
+                </plugin>
             </plugins>
         </pluginManagement>
         <plugins>
@@ -253,6 +275,10 @@
             </plugin>
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-checkstyle-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-javadoc-plugin</artifactId>
             </plugin>
             <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/api/LocalFile.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/LocalFile.java b/twill-api/src/main/java/org/apache/twill/api/LocalFile.java
index df35a3b..bcc3e13 100644
--- a/twill-api/src/main/java/org/apache/twill/api/LocalFile.java
+++ b/twill-api/src/main/java/org/apache/twill/api/LocalFile.java
@@ -17,8 +17,8 @@
  */
 package org.apache.twill.api;
 
-import javax.annotation.Nullable;
 import java.net.URI;
+import javax.annotation.Nullable;
 
 /**
  * This interface represents a local file that will be available for the container running a {@link TwillRunnable}.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
index b40682f..0bab811 100644
--- a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
@@ -20,7 +20,8 @@ package org.apache.twill.api;
 import org.apache.twill.internal.DefaultResourceSpecification;
 
 /**
- * This interface provides specifications for resource requirements including set and get methods for number of cores, amount of memory, and number of instances.
+ * This interface provides specifications for resource requirements including set and get methods
+ * for number of cores, amount of memory, and number of instances.
  */
 public interface ResourceSpecification {
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/api/TwillController.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/TwillController.java b/twill-api/src/main/java/org/apache/twill/api/TwillController.java
index a5906f4..7c5089d 100644
--- a/twill-api/src/main/java/org/apache/twill/api/TwillController.java
+++ b/twill-api/src/main/java/org/apache/twill/api/TwillController.java
@@ -17,10 +17,10 @@
  */
 package org.apache.twill.api;
 
+import com.google.common.util.concurrent.ListenableFuture;
 import org.apache.twill.api.logging.LogHandler;
 import org.apache.twill.discovery.Discoverable;
 import org.apache.twill.discovery.ServiceDiscovered;
-import com.google.common.util.concurrent.ListenableFuture;
 
 /**
  * For controlling a running application.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/api/TwillRunnableSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/TwillRunnableSpecification.java b/twill-api/src/main/java/org/apache/twill/api/TwillRunnableSpecification.java
index bbcc5d7..d161f9d 100644
--- a/twill-api/src/main/java/org/apache/twill/api/TwillRunnableSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/api/TwillRunnableSpecification.java
@@ -17,8 +17,8 @@
  */
 package org.apache.twill.api;
 
-import org.apache.twill.internal.DefaultTwillRunnableSpecification;
 import com.google.common.collect.ImmutableMap;
+import org.apache.twill.internal.DefaultTwillRunnableSpecification;
 
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/api/TwillSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/TwillSpecification.java b/twill-api/src/main/java/org/apache/twill/api/TwillSpecification.java
index 00d171d..3931d04 100644
--- a/twill-api/src/main/java/org/apache/twill/api/TwillSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/api/TwillSpecification.java
@@ -17,24 +17,24 @@
  */
 package org.apache.twill.api;
 
-import org.apache.twill.internal.DefaultLocalFile;
-import org.apache.twill.internal.DefaultRuntimeSpecification;
-import org.apache.twill.internal.DefaultTwillRunnableSpecification;
-import org.apache.twill.internal.DefaultTwillSpecification;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import org.apache.twill.internal.DefaultLocalFile;
+import org.apache.twill.internal.DefaultRuntimeSpecification;
+import org.apache.twill.internal.DefaultTwillRunnableSpecification;
+import org.apache.twill.internal.DefaultTwillSpecification;
 
-import javax.annotation.Nullable;
 import java.io.File;
 import java.net.URI;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import javax.annotation.Nullable;
 
 /**
  * Represents specification of a {@link TwillApplication}.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/internal/DefaultEventHandlerSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/DefaultEventHandlerSpecification.java b/twill-api/src/main/java/org/apache/twill/internal/DefaultEventHandlerSpecification.java
index df21400..83e7c38 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/DefaultEventHandlerSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/DefaultEventHandlerSpecification.java
@@ -17,11 +17,8 @@
  */
 package org.apache.twill.internal;
 
-import org.apache.twill.api.EventHandlerSpecification;
-import org.apache.twill.api.EventHandler;
-import org.apache.twill.api.EventHandlerSpecification;
-import org.apache.twill.api.EventHandlerSpecification;
 import com.google.common.collect.ImmutableMap;
+import org.apache.twill.api.EventHandler;
 import org.apache.twill.api.EventHandlerSpecification;
 
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/internal/DefaultLocalFile.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/DefaultLocalFile.java b/twill-api/src/main/java/org/apache/twill/internal/DefaultLocalFile.java
index e43c0c0..4fc78e1 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/DefaultLocalFile.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/DefaultLocalFile.java
@@ -19,8 +19,8 @@ package org.apache.twill.internal;
 
 import org.apache.twill.api.LocalFile;
 
-import javax.annotation.Nullable;
 import java.net.URI;
+import javax.annotation.Nullable;
 
 /**
  * A straightforward implementation of {@link LocalFile}.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceReport.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceReport.java b/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceReport.java
index c4c8a29..b6c2b73 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceReport.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceReport.java
@@ -17,11 +17,11 @@
  */
 package org.apache.twill.internal;
 
-import org.apache.twill.api.ResourceReport;
-import org.apache.twill.api.TwillRunResources;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimaps;
 import com.google.common.collect.SetMultimap;
+import org.apache.twill.api.ResourceReport;
+import org.apache.twill.api.TwillRunResources;
 
 import java.util.Collection;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/internal/DefaultRuntimeSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/DefaultRuntimeSpecification.java b/twill-api/src/main/java/org/apache/twill/internal/DefaultRuntimeSpecification.java
index c4f496e..9de81b1 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/DefaultRuntimeSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/DefaultRuntimeSpecification.java
@@ -17,11 +17,11 @@
  */
 package org.apache.twill.internal;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.twill.api.LocalFile;
 import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.api.RuntimeSpecification;
 import org.apache.twill.api.TwillRunnableSpecification;
-import com.google.common.collect.ImmutableList;
 
 import java.util.Collection;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillRunnableSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillRunnableSpecification.java b/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillRunnableSpecification.java
index 14ea7f5..0d2eb22 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillRunnableSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillRunnableSpecification.java
@@ -17,8 +17,8 @@
  */
 package org.apache.twill.internal;
 
-import org.apache.twill.api.TwillRunnableSpecification;
 import com.google.common.collect.ImmutableMap;
+import org.apache.twill.api.TwillRunnableSpecification;
 
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillSpecification.java b/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillSpecification.java
index 6bb2b15..fdb8b32 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillSpecification.java
@@ -17,18 +17,18 @@
  */
 package org.apache.twill.internal;
 
-import org.apache.twill.api.EventHandlerSpecification;
-import org.apache.twill.api.RuntimeSpecification;
-import org.apache.twill.api.TwillSpecification;
 import com.google.common.base.Objects;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
+import org.apache.twill.api.EventHandlerSpecification;
+import org.apache.twill.api.RuntimeSpecification;
+import org.apache.twill.api.TwillSpecification;
 
-import javax.annotation.Nullable;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import javax.annotation.Nullable;
 
 /**
  * Straightforward implementation of {@link org.apache.twill.api.TwillSpecification}.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-api/src/main/java/org/apache/twill/internal/RunIds.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/RunIds.java b/twill-api/src/main/java/org/apache/twill/internal/RunIds.java
index 7249d81..b7641f3 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/RunIds.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/RunIds.java
@@ -17,8 +17,8 @@
  */
 package org.apache.twill.internal;
 
-import org.apache.twill.api.RunId;
 import com.google.common.base.Preconditions;
+import org.apache.twill.api.RunId;
 
 import java.util.UUID;
 
@@ -65,7 +65,7 @@ public final class RunIds {
       if (other == null || !(other instanceof RunId)) {
         return false;
       }
-      return id.equals(((RunId)other).getId());
+      return id.equals(((RunId) other).getId());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
----------------------------------------------------------------------
diff --git a/twill-common/src/main/java/org/apache/twill/filesystem/Location.java b/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
index dee9546..a2d1f9a 100644
--- a/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
+++ b/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
@@ -17,11 +17,11 @@
  */
 package org.apache.twill.filesystem;
 
-import javax.annotation.Nullable;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.URI;
+import javax.annotation.Nullable;
 
 /**
  * This interface defines the location and operations of a resource on the filesystem.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/AbstractExecutionServiceController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/AbstractExecutionServiceController.java b/twill-core/src/main/java/org/apache/twill/internal/AbstractExecutionServiceController.java
index 974639d..acb6b7b 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/AbstractExecutionServiceController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/AbstractExecutionServiceController.java
@@ -17,13 +17,13 @@
  */
 package org.apache.twill.internal;
 
-import org.apache.twill.api.RunId;
-import org.apache.twill.api.ServiceController;
-import org.apache.twill.common.Threads;
 import com.google.common.util.concurrent.AbstractIdleService;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Service;
+import org.apache.twill.api.RunId;
+import org.apache.twill.api.ServiceController;
+import org.apache.twill.common.Threads;
 
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -31,7 +31,8 @@ import java.util.concurrent.Executor;
 import java.util.concurrent.atomic.AtomicReference;
 
 /**
- *
+ * An abstract base class for implementing {@link ServiceController} that deal with Service state transition and
+ * listener callback.
  */
 public abstract class AbstractExecutionServiceController implements ServiceController {
 
@@ -120,7 +121,7 @@ public abstract class AbstractExecutionServiceController implements ServiceContr
   }
 
   /**
-   * Inner class for dispatching listener call back to a list of listeners
+   * Inner class for dispatching listener call back to a list of listeners.
    */
   private static final class ListenerExecutors implements Listener {
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java b/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
index 71f0c14..d45a7c3 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
@@ -47,7 +47,7 @@ import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
 /**
- * A abstract base class for {@link org.apache.twill.api.TwillController} implementation that uses Zookeeper to controller a
+ * A abstract base class for {@link TwillController} implementation that uses Zookeeper to controller a
  * running twill application.
  */
 public abstract class AbstractTwillController extends AbstractZKServiceController implements TwillController {

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/AbstractZKServiceController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/AbstractZKServiceController.java b/twill-core/src/main/java/org/apache/twill/internal/AbstractZKServiceController.java
index 98cc2b8..a132128 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/AbstractZKServiceController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/AbstractZKServiceController.java
@@ -17,6 +17,12 @@
  */
 package org.apache.twill.internal;
 
+import com.google.common.base.Charsets;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.gson.GsonBuilder;
 import org.apache.twill.api.Command;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.ServiceController;
@@ -29,12 +35,6 @@ import org.apache.twill.internal.state.StateNode;
 import org.apache.twill.internal.state.SystemMessages;
 import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.ZKClient;
-import com.google.common.base.Charsets;
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.gson.GsonBuilder;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/Configs.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/Configs.java b/twill-core/src/main/java/org/apache/twill/internal/Configs.java
index 0fa1df8..f2f9c59 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/Configs.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/Configs.java
@@ -18,10 +18,13 @@
 package org.apache.twill.internal;
 
 /**
- *
+ * Defines keys and default values constants being used for configuration.
  */
 public final class Configs {
 
+  /**
+   * Defines keys being used in configuration.
+   */
   public static final class Keys {
     /**
      * Size in MB of reserved memory for Java process (non-heap memory).
@@ -32,6 +35,9 @@ public final class Configs {
     }
   }
 
+  /**
+   * Defines default configuration values.
+   */
   public static final class Defaults {
     // By default have 200MB reserved for Java process.
     public static final int JAVA_RESERVED_MEMORY_MB = 200;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/Constants.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/Constants.java b/twill-core/src/main/java/org/apache/twill/internal/Constants.java
index efe91a7..9912638 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/Constants.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/Constants.java
@@ -31,7 +31,7 @@ public final class Constants {
 
   public static final long PROVISION_TIMEOUT = 30000;
 
-  /** Memory size of AM */
+  /** Memory size of AM. */
   public static final int APP_MASTER_MEMORY_MB = 512;
 
   public static final int APP_MASTER_RESERVED_MEMORY_MB = 150;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/SingleRunnableApplication.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/SingleRunnableApplication.java b/twill-core/src/main/java/org/apache/twill/internal/SingleRunnableApplication.java
index a52afe1..2f3d0bf 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/SingleRunnableApplication.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/SingleRunnableApplication.java
@@ -24,7 +24,7 @@ import org.apache.twill.api.TwillRunnableSpecification;
 import org.apache.twill.api.TwillSpecification;
 
 /**
- * A simple {@link org.apache.twill.api.TwillApplication} that contains only one {@link org.apache.twill.api.TwillRunnable}.
+ * A simple {@link TwillApplication} that contains only one {@link TwillRunnable}.
  */
 public class SingleRunnableApplication implements TwillApplication {
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
index 8b090bd..bb46cd5 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
@@ -17,9 +17,9 @@
  */
 package org.apache.twill.internal;
 
+import com.google.common.util.concurrent.ListenableFuture;
 import org.apache.twill.api.ServiceController;
 import org.apache.twill.internal.state.Message;
-import com.google.common.util.concurrent.ListenableFuture;
 
 /**
  * A {@link ServiceController} that allows sending a message directly. Internal use only.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
index dad8cbe..5d2f33c 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
@@ -18,6 +18,7 @@
 package org.apache.twill.internal;
 
 import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
 import org.apache.twill.api.LocalFile;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.RuntimeSpecification;
@@ -27,7 +28,6 @@ import org.apache.twill.internal.state.StateNode;
 import org.apache.twill.launcher.TwillLauncher;
 import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.ZKClient;
-import com.google.common.util.concurrent.ListenableFuture;
 import org.apache.twill.zookeeper.ZKOperations;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/ZKMessages.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/ZKMessages.java b/twill-core/src/main/java/org/apache/twill/internal/ZKMessages.java
index 03575dd..b7905d9 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/ZKMessages.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/ZKMessages.java
@@ -17,14 +17,14 @@
  */
 package org.apache.twill.internal;
 
-import org.apache.twill.internal.state.Message;
-import org.apache.twill.internal.state.MessageCodec;
-import org.apache.twill.zookeeper.ZKClient;
-import org.apache.twill.zookeeper.ZKOperations;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
+import org.apache.twill.internal.state.Message;
+import org.apache.twill.internal.state.MessageCodec;
+import org.apache.twill.zookeeper.ZKClient;
+import org.apache.twill.zookeeper.ZKOperations;
 import org.apache.zookeeper.CreateMode;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/ZKServiceDecorator.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/ZKServiceDecorator.java b/twill-core/src/main/java/org/apache/twill/internal/ZKServiceDecorator.java
index d434bac..7592176 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/ZKServiceDecorator.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/ZKServiceDecorator.java
@@ -17,6 +17,21 @@
  */
 package org.apache.twill.internal;
 
+import com.google.common.base.Charsets;
+import com.google.common.base.Supplier;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.AbstractService;
+import com.google.common.util.concurrent.AsyncFunction;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.Service;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.ServiceController;
 import org.apache.twill.common.ServiceListenerAdapter;
@@ -33,21 +48,6 @@ import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.OperationFuture;
 import org.apache.twill.zookeeper.ZKClient;
 import org.apache.twill.zookeeper.ZKOperations;
-import com.google.common.base.Charsets;
-import com.google.common.base.Supplier;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.AbstractService;
-import com.google.common.util.concurrent.AsyncFunction;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.google.common.util.concurrent.Service;
-import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonObject;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -55,12 +55,12 @@ import org.apache.zookeeper.Watcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.annotation.Nullable;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import javax.annotation.Nullable;
 
 /**
  * A {@link Service} decorator that wrap another {@link Service} with the service states reflected

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/json/ArgumentsCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/ArgumentsCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/ArgumentsCodec.java
index 07d4c1d..484231f 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/ArgumentsCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/ArgumentsCodec.java
@@ -17,7 +17,6 @@
  */
 package org.apache.twill.internal.json;
 
-import org.apache.twill.internal.Arguments;
 import com.google.common.collect.ImmutableMultimap;
 import com.google.common.io.InputSupplier;
 import com.google.common.io.OutputSupplier;
@@ -31,6 +30,7 @@ import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
+import org.apache.twill.internal.Arguments;
 
 import java.io.IOException;
 import java.io.Reader;
@@ -41,7 +41,7 @@ import java.util.List;
 import java.util.Map;
 
 /**
- *
+ * Gson codec for {@link Arguments}.
  */
 public final class ArgumentsCodec implements JsonSerializer<Arguments>, JsonDeserializer<Arguments> {
 
@@ -81,9 +81,9 @@ public final class ArgumentsCodec implements JsonSerializer<Arguments>, JsonDese
   public Arguments deserialize(JsonElement json, Type typeOfT,
                               JsonDeserializationContext context) throws JsonParseException {
     JsonObject jsonObj = json.getAsJsonObject();
-    List<String> arguments = context.deserialize(jsonObj.get("arguments"), new TypeToken<List<String>>() {}.getType());
+    List<String> arguments = context.deserialize(jsonObj.get("arguments"), new TypeToken<List<String>>() { }.getType());
     Map<String, Collection<String>> args = context.deserialize(jsonObj.get("runnableArguments"),
-                                                               new TypeToken<Map<String, Collection<String>>>(){
+                                                               new TypeToken<Map<String, Collection<String>>>() {
                                                                }.getType());
 
     ImmutableMultimap.Builder<String, String> builder = ImmutableMultimap.builder();
@@ -92,4 +92,4 @@ public final class ArgumentsCodec implements JsonSerializer<Arguments>, JsonDese
     }
     return new Arguments(arguments, builder.build());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/json/LocalFileCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/LocalFileCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/LocalFileCodec.java
index 680a36c..8f2b9c4 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/LocalFileCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/LocalFileCodec.java
@@ -17,8 +17,6 @@
  */
 package org.apache.twill.internal.json;
 
-import org.apache.twill.api.LocalFile;
-import org.apache.twill.internal.DefaultLocalFile;
 import com.google.gson.JsonDeserializationContext;
 import com.google.gson.JsonDeserializer;
 import com.google.gson.JsonElement;
@@ -26,6 +24,8 @@ import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
+import org.apache.twill.api.LocalFile;
+import org.apache.twill.internal.DefaultLocalFile;
 
 import java.lang.reflect.Type;
 import java.net.URI;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportAdapter.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportAdapter.java b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportAdapter.java
index e473fe7..4427704 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportAdapter.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportAdapter.java
@@ -17,10 +17,10 @@
  */
 package org.apache.twill.internal.json;
 
-import org.apache.twill.api.ResourceReport;
-import org.apache.twill.api.TwillRunResources;
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
+import org.apache.twill.api.ResourceReport;
+import org.apache.twill.api.TwillRunResources;
 
 import java.io.Reader;
 import java.io.Writer;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportCodec.java
index 884d889..89a8433 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceReportCodec.java
@@ -17,9 +17,6 @@
  */
 package org.apache.twill.internal.json;
 
-import org.apache.twill.api.ResourceReport;
-import org.apache.twill.api.TwillRunResources;
-import org.apache.twill.internal.DefaultResourceReport;
 import com.google.gson.JsonDeserializationContext;
 import com.google.gson.JsonDeserializer;
 import com.google.gson.JsonElement;
@@ -28,6 +25,9 @@ import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
 import com.google.gson.reflect.TypeToken;
+import org.apache.twill.api.ResourceReport;
+import org.apache.twill.api.TwillRunResources;
+import org.apache.twill.internal.DefaultResourceReport;
 
 import java.lang.reflect.Type;
 import java.util.Collection;
@@ -45,9 +45,9 @@ public final class ResourceReportCodec implements JsonSerializer<ResourceReport>
 
     json.addProperty("appMasterId", src.getApplicationId());
     json.add("appMasterResources", context.serialize(
-      src.getAppMasterResources(), new TypeToken<TwillRunResources>(){}.getType()));
+      src.getAppMasterResources(), new TypeToken<TwillRunResources>() { }.getType()));
     json.add("runnableResources", context.serialize(
-      src.getResources(), new TypeToken<Map<String, Collection<TwillRunResources>>>(){}.getType()));
+      src.getResources(), new TypeToken<Map<String, Collection<TwillRunResources>>>() { }.getType()));
 
     return json;
   }
@@ -60,7 +60,7 @@ public final class ResourceReportCodec implements JsonSerializer<ResourceReport>
     TwillRunResources masterResources = context.deserialize(
       jsonObj.get("appMasterResources"), TwillRunResources.class);
     Map<String, Collection<TwillRunResources>> resources = context.deserialize(
-      jsonObj.get("runnableResources"), new TypeToken<Map<String, Collection<TwillRunResources>>>(){}.getType());
+      jsonObj.get("runnableResources"), new TypeToken<Map<String, Collection<TwillRunResources>>>() { }.getType());
 
     return new DefaultResourceReport(appMasterId, masterResources, resources);
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
index bea73c4..d3b9707 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
@@ -17,8 +17,6 @@
  */
 package org.apache.twill.internal.json;
 
-import org.apache.twill.api.ResourceSpecification;
-import org.apache.twill.internal.DefaultResourceSpecification;
 import com.google.gson.JsonDeserializationContext;
 import com.google.gson.JsonDeserializer;
 import com.google.gson.JsonElement;
@@ -26,6 +24,8 @@ import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
+import org.apache.twill.api.ResourceSpecification;
+import org.apache.twill.internal.DefaultResourceSpecification;
 
 import java.lang.reflect.Type;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/json/RuntimeSpecificationCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/RuntimeSpecificationCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/RuntimeSpecificationCodec.java
index 867f4a8..970152e 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/RuntimeSpecificationCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/RuntimeSpecificationCodec.java
@@ -17,11 +17,6 @@
  */
 package org.apache.twill.internal.json;
 
-import org.apache.twill.api.LocalFile;
-import org.apache.twill.api.ResourceSpecification;
-import org.apache.twill.api.RuntimeSpecification;
-import org.apache.twill.api.TwillRunnableSpecification;
-import org.apache.twill.internal.DefaultRuntimeSpecification;
 import com.google.common.reflect.TypeToken;
 import com.google.gson.JsonDeserializationContext;
 import com.google.gson.JsonDeserializer;
@@ -30,12 +25,17 @@ import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
+import org.apache.twill.api.LocalFile;
+import org.apache.twill.api.ResourceSpecification;
+import org.apache.twill.api.RuntimeSpecification;
+import org.apache.twill.api.TwillRunnableSpecification;
+import org.apache.twill.internal.DefaultRuntimeSpecification;
 
 import java.lang.reflect.Type;
 import java.util.Collection;
 
 /**
- *
+ * Gson codec for {@link RuntimeSpecification}.
  */
 final class RuntimeSpecificationCodec implements JsonSerializer<RuntimeSpecification>,
                                                  JsonDeserializer<RuntimeSpecification> {
@@ -46,7 +46,7 @@ final class RuntimeSpecificationCodec implements JsonSerializer<RuntimeSpecifica
     json.addProperty("name", src.getName());
     json.add("runnable", context.serialize(src.getRunnableSpecification(), TwillRunnableSpecification.class));
     json.add("resources", context.serialize(src.getResourceSpecification(), ResourceSpecification.class));
-    json.add("files", context.serialize(src.getLocalFiles(), new TypeToken<Collection<LocalFile>>(){}.getType()));
+    json.add("files", context.serialize(src.getLocalFiles(), new TypeToken<Collection<LocalFile>>() { }.getType()));
 
     return json;
   }
@@ -62,7 +62,7 @@ final class RuntimeSpecificationCodec implements JsonSerializer<RuntimeSpecifica
     ResourceSpecification resources = context.deserialize(jsonObj.get("resources"),
                                                           ResourceSpecification.class);
     Collection<LocalFile> files = context.deserialize(jsonObj.get("files"),
-                                                      new TypeToken<Collection<LocalFile>>(){}.getType());
+                                                      new TypeToken<Collection<LocalFile>>() { }.getType());
 
     return new DefaultRuntimeSpecification(name, runnable, resources, files);
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/json/StateNodeCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/StateNodeCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/StateNodeCodec.java
index c1e9d1c..f620231 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/StateNodeCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/StateNodeCodec.java
@@ -17,8 +17,6 @@
  */
 package org.apache.twill.internal.json;
 
-import org.apache.twill.api.ServiceController;
-import org.apache.twill.internal.state.StateNode;
 import com.google.gson.JsonDeserializationContext;
 import com.google.gson.JsonDeserializer;
 import com.google.gson.JsonElement;
@@ -26,11 +24,13 @@ import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
+import org.apache.twill.api.ServiceController;
+import org.apache.twill.internal.state.StateNode;
 
 import java.lang.reflect.Type;
 
 /**
- *
+ * Gson codec for {@link StateNode}.
  */
 public final class StateNodeCodec implements JsonSerializer<StateNode>, JsonDeserializer<StateNode> {
 
@@ -42,7 +42,8 @@ public final class StateNodeCodec implements JsonSerializer<StateNode>, JsonDese
     String errorMessage = jsonObj.has("errorMessage") ? jsonObj.get("errorMessage").getAsString() : null;
 
     return new StateNode(state, errorMessage,
-                         context.<StackTraceElement[]>deserialize(jsonObj.get("stackTraces"), StackTraceElement[].class));
+                         context.<StackTraceElement[]>deserialize(jsonObj.get("stackTraces"),
+                                                                  StackTraceElement[].class));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunResourcesCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunResourcesCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunResourcesCodec.java
index 8951173..c39fa16 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunResourcesCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunResourcesCodec.java
@@ -17,8 +17,6 @@
  */
 package org.apache.twill.internal.json;
 
-import org.apache.twill.api.TwillRunResources;
-import org.apache.twill.internal.DefaultTwillRunResources;
 import com.google.gson.JsonDeserializationContext;
 import com.google.gson.JsonDeserializer;
 import com.google.gson.JsonElement;
@@ -26,6 +24,8 @@ import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
+import org.apache.twill.api.TwillRunResources;
+import org.apache.twill.internal.DefaultTwillRunResources;
 
 import java.lang.reflect.Type;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunnableSpecificationCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunnableSpecificationCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunnableSpecificationCodec.java
index f37c1e8..c5c586a 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunnableSpecificationCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunnableSpecificationCodec.java
@@ -17,8 +17,6 @@
  */
 package org.apache.twill.internal.json;
 
-import org.apache.twill.api.TwillRunnableSpecification;
-import org.apache.twill.internal.DefaultTwillRunnableSpecification;
 import com.google.common.reflect.TypeToken;
 import com.google.gson.JsonDeserializationContext;
 import com.google.gson.JsonDeserializer;
@@ -27,12 +25,14 @@ import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
+import org.apache.twill.api.TwillRunnableSpecification;
+import org.apache.twill.internal.DefaultTwillRunnableSpecification;
 
 import java.lang.reflect.Type;
 import java.util.Map;
 
 /**
- *
+ * Gson codec for {@link TwillRunnableSpecification}.
  */
 final class TwillRunnableSpecificationCodec implements JsonSerializer<TwillRunnableSpecification>,
                                                        JsonDeserializer<TwillRunnableSpecification> {
@@ -43,7 +43,7 @@ final class TwillRunnableSpecificationCodec implements JsonSerializer<TwillRunna
 
     json.addProperty("classname", src.getClassName());
     json.addProperty("name", src.getName());
-    json.add("arguments", context.serialize(src.getConfigs(), new TypeToken<Map<String, String>>(){}.getType()));
+    json.add("arguments", context.serialize(src.getConfigs(), new TypeToken<Map<String, String>>() { }.getType()));
 
     return json;
   }
@@ -56,7 +56,7 @@ final class TwillRunnableSpecificationCodec implements JsonSerializer<TwillRunna
     String className = jsonObj.get("classname").getAsString();
     String name = jsonObj.get("name").getAsString();
     Map<String, String> arguments = context.deserialize(jsonObj.get("arguments"),
-                                                        new TypeToken<Map<String, String>>(){}.getType());
+                                                        new TypeToken<Map<String, String>>() { }.getType());
 
     return new DefaultTwillRunnableSpecification(className, name, arguments);
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/json/TwillSpecificationCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/TwillSpecificationCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/TwillSpecificationCodec.java
index 5d88350..d46434a 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/TwillSpecificationCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/TwillSpecificationCodec.java
@@ -17,11 +17,6 @@
  */
 package org.apache.twill.internal.json;
 
-import org.apache.twill.api.EventHandlerSpecification;
-import org.apache.twill.api.RuntimeSpecification;
-import org.apache.twill.api.TwillSpecification;
-import org.apache.twill.internal.DefaultEventHandlerSpecification;
-import org.apache.twill.internal.DefaultTwillSpecification;
 import com.google.common.reflect.TypeToken;
 import com.google.gson.JsonDeserializationContext;
 import com.google.gson.JsonDeserializer;
@@ -30,6 +25,11 @@ import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
+import org.apache.twill.api.EventHandlerSpecification;
+import org.apache.twill.api.RuntimeSpecification;
+import org.apache.twill.api.TwillSpecification;
+import org.apache.twill.internal.DefaultEventHandlerSpecification;
+import org.apache.twill.internal.DefaultTwillSpecification;
 
 import java.lang.reflect.Type;
 import java.util.List;
@@ -47,9 +47,9 @@ final class TwillSpecificationCodec implements JsonSerializer<TwillSpecification
     JsonObject json = new JsonObject();
     json.addProperty("name", src.getName());
     json.add("runnables", context.serialize(src.getRunnables(),
-                                            new TypeToken<Map<String, RuntimeSpecification>>(){}.getType()));
+                                            new TypeToken<Map<String, RuntimeSpecification>>() { }.getType()));
     json.add("orders", context.serialize(src.getOrders(),
-                                         new TypeToken<List<TwillSpecification.Order>>(){}.getType()));
+                                         new TypeToken<List<TwillSpecification.Order>>() { }.getType()));
     EventHandlerSpecification eventHandler = src.getEventHandler();
     if (eventHandler != null) {
       json.add("handler", context.serialize(eventHandler, EventHandlerSpecification.class));
@@ -65,9 +65,9 @@ final class TwillSpecificationCodec implements JsonSerializer<TwillSpecification
 
     String name = jsonObj.get("name").getAsString();
     Map<String, RuntimeSpecification> runnables = context.deserialize(
-      jsonObj.get("runnables"), new TypeToken<Map<String, RuntimeSpecification>>(){}.getType());
+      jsonObj.get("runnables"), new TypeToken<Map<String, RuntimeSpecification>>() { }.getType());
     List<TwillSpecification.Order> orders = context.deserialize(
-      jsonObj.get("orders"), new TypeToken<List<TwillSpecification.Order>>(){}.getType());
+      jsonObj.get("orders"), new TypeToken<List<TwillSpecification.Order>>() { }.getType());
 
     JsonElement handler = jsonObj.get("handler");
     EventHandlerSpecification eventHandler = null;
@@ -84,7 +84,7 @@ final class TwillSpecificationCodec implements JsonSerializer<TwillSpecification
     @Override
     public JsonElement serialize(TwillSpecification.Order src, Type typeOfSrc, JsonSerializationContext context) {
       JsonObject json = new JsonObject();
-      json.add("names", context.serialize(src.getNames(), new TypeToken<Set<String>>(){}.getType()));
+      json.add("names", context.serialize(src.getNames(), new TypeToken<Set<String>>() { }.getType()));
       json.addProperty("type", src.getType().name());
       return json;
     }
@@ -94,7 +94,7 @@ final class TwillSpecificationCodec implements JsonSerializer<TwillSpecification
                                                 JsonDeserializationContext context) throws JsonParseException {
       JsonObject jsonObj = json.getAsJsonObject();
 
-      Set<String> names = context.deserialize(jsonObj.get("names"), new TypeToken<Set<String>>(){}.getType());
+      Set<String> names = context.deserialize(jsonObj.get("names"), new TypeToken<Set<String>>() { }.getType());
       TwillSpecification.Order.Type type = TwillSpecification.Order.Type.valueOf(jsonObj.get("type").getAsString());
 
       return new DefaultTwillSpecification.DefaultOrder(names, type);
@@ -108,7 +108,7 @@ final class TwillSpecificationCodec implements JsonSerializer<TwillSpecification
     public JsonElement serialize(EventHandlerSpecification src, Type typeOfSrc, JsonSerializationContext context) {
       JsonObject json = new JsonObject();
       json.addProperty("classname", src.getClassName());
-      json.add("configs", context.serialize(src.getConfigs(), new TypeToken<Map<String, String>>(){}.getType()));
+      json.add("configs", context.serialize(src.getConfigs(), new TypeToken<Map<String, String>>() { }.getType()));
       return json;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/kafka/EmbeddedKafkaServer.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/EmbeddedKafkaServer.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/EmbeddedKafkaServer.java
index d82d617..0beab44 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/EmbeddedKafkaServer.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/EmbeddedKafkaServer.java
@@ -21,7 +21,6 @@ import com.google.common.util.concurrent.AbstractIdleService;
 import kafka.server.KafkaConfig;
 import kafka.server.KafkaServerStartable;
 
-import java.io.File;
 import java.util.Properties;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/logging/LogEntryDecoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/logging/LogEntryDecoder.java b/twill-core/src/main/java/org/apache/twill/internal/logging/LogEntryDecoder.java
index dc11666..9bb0896 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/logging/LogEntryDecoder.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/logging/LogEntryDecoder.java
@@ -17,13 +17,13 @@
  */
 package org.apache.twill.internal.logging;
 
-import org.apache.twill.api.logging.LogEntry;
-import org.apache.twill.internal.json.JsonUtils;
 import com.google.gson.JsonDeserializationContext;
 import com.google.gson.JsonDeserializer;
 import com.google.gson.JsonElement;
 import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
+import org.apache.twill.api.logging.LogEntry;
+import org.apache.twill.internal.json.JsonUtils;
 
 import java.lang.reflect.Type;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/state/MessageCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/state/MessageCodec.java b/twill-core/src/main/java/org/apache/twill/internal/state/MessageCodec.java
index 176f620..60c5da1 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/state/MessageCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/state/MessageCodec.java
@@ -17,7 +17,6 @@
  */
 package org.apache.twill.internal.state;
 
-import org.apache.twill.api.Command;
 import com.google.common.base.Charsets;
 import com.google.common.reflect.TypeToken;
 import com.google.gson.Gson;
@@ -29,16 +28,17 @@ import com.google.gson.JsonObject;
 import com.google.gson.JsonParseException;
 import com.google.gson.JsonSerializationContext;
 import com.google.gson.JsonSerializer;
+import org.apache.twill.api.Command;
 
 import java.lang.reflect.Type;
 import java.util.Map;
 
 /**
- *
+ * Gson codec for {@link Message}.
  */
 public final class MessageCodec {
 
-  private static final Type OPTIONS_TYPE = new TypeToken<Map<String, String>>() {}.getType();
+  private static final Type OPTIONS_TYPE = new TypeToken<Map<String, String>>() { }.getType();
   private static final Gson GSON = new GsonBuilder()
                                         .registerTypeAdapter(Message.class, new MessageAdapter())
                                         .registerTypeAdapter(Command.class, new CommandAdapter())

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/state/SimpleMessage.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/state/SimpleMessage.java b/twill-core/src/main/java/org/apache/twill/internal/state/SimpleMessage.java
index e146e56..9cfbe29 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/state/SimpleMessage.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/state/SimpleMessage.java
@@ -17,8 +17,8 @@
  */
 package org.apache.twill.internal.state;
 
-import org.apache.twill.api.Command;
 import com.google.common.base.Objects;
+import org.apache.twill.api.Command;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/state/StateNode.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/state/StateNode.java b/twill-core/src/main/java/org/apache/twill/internal/state/StateNode.java
index d66f8a2..6cdab48 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/state/StateNode.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/state/StateNode.java
@@ -17,8 +17,8 @@
  */
 package org.apache.twill.internal.state;
 
-import org.apache.twill.api.ServiceController;
 import com.google.common.util.concurrent.Service;
+import org.apache.twill.api.ServiceController;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/internal/state/SystemMessages.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/state/SystemMessages.java b/twill-core/src/main/java/org/apache/twill/internal/state/SystemMessages.java
index 9877121..73683bd 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/state/SystemMessages.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/state/SystemMessages.java
@@ -17,8 +17,8 @@
  */
 package org.apache.twill.internal.state;
 
-import org.apache.twill.api.Command;
 import com.google.common.base.Preconditions;
+import org.apache.twill.api.Command;
 
 /**
  * Collection of predefined system messages.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaPublisher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaPublisher.java b/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaPublisher.java
index bffce97..0be45cb 100644
--- a/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaPublisher.java
+++ b/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaPublisher.java
@@ -38,7 +38,7 @@ public interface KafkaPublisher {
      *                ByteBuffer would be {@code 0}.
      * @param partitionKey Key for computing the partition Id to publish to. The {@link Object#hashCode()} method
      *                     will be invoke to compute the id.
-     * @return
+     * @return This {@link Preparer} instance.
      */
     Preparer add(ByteBuffer message, Object partitionKey);
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java b/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
index 382dc95..048e489 100644
--- a/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
+++ b/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
@@ -17,6 +17,11 @@
  */
 package org.apache.twill.internal;
 
+import com.google.common.base.Suppliers;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.AbstractIdleService;
+import com.google.common.util.concurrent.Service;
+import com.google.gson.JsonObject;
 import org.apache.twill.api.Command;
 import org.apache.twill.api.ResourceReport;
 import org.apache.twill.api.RunId;
@@ -30,11 +35,6 @@ import org.apache.twill.internal.zookeeper.InMemoryZKServer;
 import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.ZKClient;
 import org.apache.twill.zookeeper.ZKClientService;
-import com.google.common.base.Suppliers;
-import com.google.common.collect.ImmutableList;
-import com.google.common.util.concurrent.AbstractIdleService;
-import com.google.common.util.concurrent.Service;
-import com.google.gson.JsonObject;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/test/java/org/apache/twill/internal/state/MessageCodecTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/state/MessageCodecTest.java b/twill-core/src/test/java/org/apache/twill/internal/state/MessageCodecTest.java
index d267cf8..2090cca 100644
--- a/twill-core/src/test/java/org/apache/twill/internal/state/MessageCodecTest.java
+++ b/twill-core/src/test/java/org/apache/twill/internal/state/MessageCodecTest.java
@@ -17,8 +17,8 @@
  */
 package org.apache.twill.internal.state;
 
-import org.apache.twill.api.Command;
 import com.google.common.collect.ImmutableMap;
+import org.apache.twill.api.Command;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/test/java/org/apache/twill/internal/state/ZKServiceDecoratorTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/state/ZKServiceDecoratorTest.java b/twill-core/src/test/java/org/apache/twill/internal/state/ZKServiceDecoratorTest.java
index 47d8562..401d6e3 100644
--- a/twill-core/src/test/java/org/apache/twill/internal/state/ZKServiceDecoratorTest.java
+++ b/twill-core/src/test/java/org/apache/twill/internal/state/ZKServiceDecoratorTest.java
@@ -17,13 +17,6 @@
  */
 package org.apache.twill.internal.state;
 
-import org.apache.twill.api.RunId;
-import org.apache.twill.internal.RunIds;
-import org.apache.twill.internal.ZKServiceDecorator;
-import org.apache.twill.internal.zookeeper.InMemoryZKServer;
-import org.apache.twill.zookeeper.NodeData;
-import org.apache.twill.zookeeper.ZKClientService;
-import org.apache.twill.zookeeper.ZKClients;
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
@@ -35,6 +28,13 @@ import com.google.common.util.concurrent.Service;
 import com.google.gson.Gson;
 import com.google.gson.JsonElement;
 import com.google.gson.JsonObject;
+import org.apache.twill.api.RunId;
+import org.apache.twill.internal.RunIds;
+import org.apache.twill.internal.ZKServiceDecorator;
+import org.apache.twill.internal.zookeeper.InMemoryZKServer;
+import org.apache.twill.zookeeper.NodeData;
+import org.apache.twill.zookeeper.ZKClientService;
+import org.apache.twill.zookeeper.ZKClients;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-core/src/test/java/org/apache/twill/internal/utils/ApplicationBundlerTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/utils/ApplicationBundlerTest.java b/twill-core/src/test/java/org/apache/twill/internal/utils/ApplicationBundlerTest.java
index 508cadb..38d0a05 100644
--- a/twill-core/src/test/java/org/apache/twill/internal/utils/ApplicationBundlerTest.java
+++ b/twill-core/src/test/java/org/apache/twill/internal/utils/ApplicationBundlerTest.java
@@ -17,13 +17,13 @@
  */
 package org.apache.twill.internal.utils;
 
-import org.apache.twill.filesystem.LocalLocationFactory;
-import org.apache.twill.filesystem.Location;
-import org.apache.twill.internal.ApplicationBundler;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.io.ByteStreams;
 import com.google.common.io.Files;
+import org.apache.twill.filesystem.LocalLocationFactory;
+import org.apache.twill.filesystem.Location;
+import org.apache.twill.internal.ApplicationBundler;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-discovery-core/src/main/java/org/apache/twill/discovery/DefaultServiceDiscovered.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/main/java/org/apache/twill/discovery/DefaultServiceDiscovered.java b/twill-discovery-core/src/main/java/org/apache/twill/discovery/DefaultServiceDiscovered.java
index 49fb641..9734683 100644
--- a/twill-discovery-core/src/main/java/org/apache/twill/discovery/DefaultServiceDiscovered.java
+++ b/twill-discovery-core/src/main/java/org/apache/twill/discovery/DefaultServiceDiscovered.java
@@ -17,9 +17,9 @@
  */
 package org.apache.twill.discovery;
 
-import org.apache.twill.common.Cancellable;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
+import org.apache.twill.common.Cancellable;
 
 import java.util.Iterator;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-discovery-core/src/main/java/org/apache/twill/discovery/InMemoryDiscoveryService.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/main/java/org/apache/twill/discovery/InMemoryDiscoveryService.java b/twill-discovery-core/src/main/java/org/apache/twill/discovery/InMemoryDiscoveryService.java
index 2f950b8..6a71855 100644
--- a/twill-discovery-core/src/main/java/org/apache/twill/discovery/InMemoryDiscoveryService.java
+++ b/twill-discovery-core/src/main/java/org/apache/twill/discovery/InMemoryDiscoveryService.java
@@ -17,10 +17,10 @@
  */
 package org.apache.twill.discovery;
 
-import org.apache.twill.common.Cancellable;
 import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Maps;
 import com.google.common.collect.SetMultimap;
+import org.apache.twill.common.Cancellable;
 
 import java.util.Map;
 import java.util.concurrent.locks.Lock;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-discovery-core/src/main/java/org/apache/twill/discovery/ZKDiscoveryService.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/main/java/org/apache/twill/discovery/ZKDiscoveryService.java b/twill-discovery-core/src/main/java/org/apache/twill/discovery/ZKDiscoveryService.java
index 0543626..f29d986 100644
--- a/twill-discovery-core/src/main/java/org/apache/twill/discovery/ZKDiscoveryService.java
+++ b/twill-discovery-core/src/main/java/org/apache/twill/discovery/ZKDiscoveryService.java
@@ -17,14 +17,6 @@
  */
 package org.apache.twill.discovery;
 
-import org.apache.twill.common.Cancellable;
-import org.apache.twill.common.Threads;
-import org.apache.twill.zookeeper.NodeChildren;
-import org.apache.twill.zookeeper.NodeData;
-import org.apache.twill.zookeeper.OperationFuture;
-import org.apache.twill.zookeeper.ZKClient;
-import org.apache.twill.zookeeper.ZKClients;
-import org.apache.twill.zookeeper.ZKOperations;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -37,6 +29,14 @@ import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
+import org.apache.twill.common.Cancellable;
+import org.apache.twill.common.Threads;
+import org.apache.twill.zookeeper.NodeChildren;
+import org.apache.twill.zookeeper.NodeData;
+import org.apache.twill.zookeeper.OperationFuture;
+import org.apache.twill.zookeeper.ZKClient;
+import org.apache.twill.zookeeper.ZKClients;
+import org.apache.twill.zookeeper.ZKOperations;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -75,13 +75,13 @@ import java.util.concurrent.locks.ReentrantLock;
  *        &#64;Override
  *        public String getName() {
  *          return 'service-name';
- *        &#125;
+ *        }
  *
  *        &#64;Override
  *        public InetSocketAddress getSocketAddress() {
  *          return new InetSocketAddress(hostname, port);
- *        &#125;
- *      &#125;);
+ *        }
+ *      });
  *      ...
  *      ...
  *      ServiceDiscovered services = service.discovery("service-name");


[07/50] [abbrv] git commit: [TWILL-27] Turn individual file log to TRACE level while building application bundler.

Posted by ch...@apache.org.
[TWILL-27] Turn individual file log to TRACE level while building application bundler.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 979634c84a1fbe71ee8bb7e73fb74e46369da1a5
Parents: fcbe54d
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Jan 27 12:36:55 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Jan 27 13:40:38 2014 -0800

----------------------------------------------------------------------
 .../java/org/apache/twill/internal/ApplicationBundler.java   | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/979634c8/twill-core/src/main/java/org/apache/twill/internal/ApplicationBundler.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/ApplicationBundler.java b/twill-core/src/main/java/org/apache/twill/internal/ApplicationBundler.java
index a0e9a71..dcc4dbd 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/ApplicationBundler.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/ApplicationBundler.java
@@ -17,8 +17,6 @@
  */
 package org.apache.twill.internal;
 
-import org.apache.twill.filesystem.Location;
-import org.apache.twill.internal.utils.Dependencies;
 import com.google.common.base.Function;
 import com.google.common.base.Splitter;
 import com.google.common.base.Throwables;
@@ -29,6 +27,8 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.io.ByteStreams;
 import com.google.common.io.Files;
+import org.apache.twill.filesystem.Location;
+import org.apache.twill.internal.utils.Dependencies;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -258,10 +258,10 @@ public final class ApplicationBundler {
    * Saves a class entry to the jar output.
    */
   private void saveEntry(String entry, URL url, Set<String> entries, JarOutputStream jarOut, boolean compress) {
-    LOG.debug("adding bundle entry " + entry);
     if (!entries.add(entry)) {
       return;
     }
+    LOG.trace("adding bundle entry " + entry);
     try {
       JarEntry jarEntry = new JarEntry(entry);
       InputStream is = url.openStream();
@@ -299,7 +299,7 @@ public final class ApplicationBundler {
    */
   private void copyDir(File baseDir, String entryPrefix,
                        Set<String> entries, JarOutputStream jarOut) throws IOException {
-    LOG.debug("adding whole dir {} to bundle at '{}'", baseDir, entryPrefix);
+    LOG.trace("adding whole dir {} to bundle at '{}'", baseDir, entryPrefix);
     URI baseUri = baseDir.toURI();
     Queue<File> queue = Lists.newLinkedList();
     Collections.addAll(queue, baseDir.listFiles());


[32/50] [abbrv] git commit: made changes as per review #18209 for #TWILL-40 rev2

Posted by ch...@apache.org.
made changes as per review #18209 for #TWILL-40 rev2

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 647880ef0d21ae5ea80f5b562bacdcd4929db8a8
Parents: e3ccab1
Author: Fabian Murariu <mu...@gmail.com>
Authored: Thu Feb 20 12:57:06 2014 +0200
Committer: Terence Yim <te...@continuuity.com>
Committed: Fri Feb 21 12:33:26 2014 -0800

----------------------------------------------------------------------
 .../java/org/apache/twill/api/ResourceSpecification.java    | 9 +++++----
 .../apache/twill/internal/DefaultResourceSpecification.java | 9 +++++----
 2 files changed, 10 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/647880ef/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
index b37e491..7600041 100644
--- a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
@@ -17,6 +17,7 @@
  */
 package org.apache.twill.api;
 
+import com.google.common.collect.Iterables;
 import org.apache.twill.internal.DefaultResourceSpecification;
 
 import java.util.Arrays;
@@ -163,9 +164,9 @@ public interface ResourceSpecification {
         return new Done();
       }
 
-      public Done setRacks(Collection<String> racks){
+      public Done setRacks(Iterable<String> racks){
         if (racks != null){
-          Builder.this.racks.addAll(racks);
+          Iterables.addAll(Builder.this.racks, racks);
         }
         return new Done();
       }
@@ -179,9 +180,9 @@ public interface ResourceSpecification {
         return new AfterHosts();
       }
 
-      public AfterHosts setHosts(Collection<String> hosts){
+      public AfterHosts setHosts(Iterable<String> hosts){
         if (hosts != null){
-          Builder.this.hosts.addAll(hosts);
+          Iterables.addAll(Builder.this.hosts, hosts);
         }
         return new AfterHosts();
       }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/647880ef/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java b/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
index 3b1cc26..2998165 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
@@ -17,6 +17,7 @@
  */
 package org.apache.twill.internal;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.twill.api.ResourceSpecification;
 
 import java.util.Collections;
@@ -47,8 +48,8 @@ public final class DefaultResourceSpecification implements ResourceSpecification
     this.instances = instances;
     this.uplink = uplink;
     this.downlink = downlink;
-    this.hosts = hosts;
-    this.racks = racks;
+    this.hosts = ImmutableList.copyOf(hosts);
+    this.racks = ImmutableList.copyOf(racks);
   }
 
   @Deprecated
@@ -74,12 +75,12 @@ public final class DefaultResourceSpecification implements ResourceSpecification
 
   @Override
   public List<String> getHosts() {
-    return Collections.unmodifiableList(this.hosts);
+    return this.hosts;
   }
 
   @Override
   public List<String> getRacks() {
-    return Collections.unmodifiableList(this.racks);
+    return this.racks;
   }
 
   @Override


[09/50] [abbrv] git commit: Added KEYS file for release.

Posted by ch...@apache.org.
Added KEYS file for release.

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

Branch: refs/heads/site
Commit: cb09429ca32e0d249f2b989522de8c935f507e22
Parents: c49b2f6
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Jan 27 14:06:30 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Jan 27 14:06:30 2014 -0800

----------------------------------------------------------------------
 KEYS | 61 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 61 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/cb09429c/KEYS
----------------------------------------------------------------------
diff --git a/KEYS b/KEYS
new file mode 100644
index 0000000..880bd27
--- /dev/null
+++ b/KEYS
@@ -0,0 +1,61 @@
+This file contains the PGP keys of various developers.
+
+Users: pgp < KEYS
+       gpg --import KEYS
+Developers: 
+        pgp -kxa <your name> and append it to this file.
+        (pgpk -ll <your name> && pgpk -xa <your name>) >> this file.
+        (gpg --list-sigs <your name>
+             && gpg --armor --export <your name>) >> this file.
+
+Type bits/keyID     Date       User ID
+pub  4096R/2167BDB4 2014-01-27 Terence Yim (CODE SIGNING KEY) <ch...@apache.org>
+
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+Version: SKS 1.1.4
+Comment: Hostname: pgp.mit.edu
+
+mQINBFLm1Z8BEADthx0oPr1A25xZ0lvdBOcqg6cPJG1UIyIBWojOrles03EBohq9p/epoiwi
+RGI2cbChLHUNEOl4fyDbBQXQsVDzAfICSjBgiqTwf7WxNH6kO1HFfV32tA6aszqBOv4iF8L6
+/GYotrf/crQgu2nc46oYuEfkKSlsroRzJxcT73vX7E1mZ+pcNLm+ZpKccT2pMZ+P3CN9m7dt
+fqYOATYp5eM9c0/RsRjg02BPOl2LAc5SRrf7jTwHJRTpqRueenxujHqT+oSIiSMOYfjGc56Q
+y9BKsBsMraM9DdccNpjvF1vlefONPXO8p3lvfxdC5pjvBuuROo64HmaR3RpGJ/4PS9RYgYkW
+TS3G9dpQM2PyE5kJ8cvESVn6YRAuR2mSQDUQcEYYG6qU2czFkzXoUBzu/Uj2iIVqZFhaS496
+ECz+3vokaGhXgESmXNaCzBpQs4SJtgANfnlF4TlnKifbbPtTyJdCnB8R/7ymd875CwFAoFgk
+Q7Ksxnb18NQJ00p0DgSV4pGtVHmz3S1dMmRqN/9w1lRccwwI7GDkSNqWciWw7DeJHJKEDUCw
+zVRpC3s38LZNscj1W6rKhCkx43EZYORAEGqVKtql6q/8SBYKmtoUAeeMPLm6akzRZBLqApyo
+Wai6zSVxbGX8e58M419/X5O/maUpohebr+apBug5TOcq5U24TwARAQABtDJUZXJlbmNlIFlp
+bSAoQ09ERSBTSUdOSU5HIEtFWSkgPGNodHlpbUBhcGFjaGUub3JnPokCNwQTAQIAIQIbAwIe
+AQIXgAUCUubWOQULCQgHAwUVCgkICwUWAgMBAAAKCRDKlsLqIWe9tHy6D/9GLQjklfAcThae
+mbe+P1GMXBSiWAaxPvRj+lfYs1NMamqgweeiGLfLWCpi5l90J1u9wya0RxCBxPYz1Ad5zb8u
+JapQzh+CcVy2ggm4CjZmlnOzLKtnZhRVIHK2O1O44HtJNdUcxJQhPvNB9qWjGKvX8e0WISml
+LLYyWvqp7jVwyaTivP87ye7n07M6A/sKFWob/mUOx/4GqEWLPTZ8Qi3v/UsAqElfk4NfaZM+
+Qc+xH17RlBo5CD9+UwD27loPV25N9PdEWOahFTR0a1VazojXiGzD/RS2srnytddK5dDe/hCy
+JNXZ4N8zoiCYxT9oSxK/UYeB9xe92mLf2YwcxYjV6sztOf1xqhEx/xFh7Yq5B3URKL0j2FA8
+Pw2H2mX+wcwZXoyfgr6QPtzAHm+ckMHVf6DNVKIWoPVwI6FKGlbK6gMxp40OLT+eYXt8/AP7
+uH6e8WJf5es4Nz4WFZ5ssIFYK/CsHOe3/7NR3s90Pszdk2ODwbEsJdHlhX1h8Uun4tH2Sj29
+gWhUGyse2bdxFfuyWzZedrUyC/4TSTgYycwmHKuviMBgmr6BQjV3IstV9MzorhNfUEx3KSJd
+HK3C/uggNvkSpUOFdyeJvt3s0Ntv7ACZvboFgb0SOXYwH1csdSObNIDamsjM3uyjiQnMc6O4
+wrEG11LCVgYxOjAc7BxqPbkCDQRS5tWfARAAuyY6r/fc8Ff7qaV6yrDoTfGc+ZRtgfgIvhIr
+qadyQFo+4s76YmV0n+8kmvzxn3ar/w+InA0rA/z9x2Cn40qgtm/CGPPJN+RL79Fm+9Ot2ZRJ
+xjBFnE0upZbvNFttgkjiBdAyLjsrxl+pcIerFhwP/l0jF4SAWB1FwYi3aZhIwZG9i27nkVsa
+neQivLFT3aZkNALHXiapHGOKx7iTf9St0UA7YVyS5ucVS0U50wqmxzn5NaWHZNF2qcHNgrlk
+N+cZIUQVccvdY6Ho/DcZ28+9msieL5GDVOnm0dl0OyFa0/i3oe12GcH1I7UDQzAm5uSUy2Jf
+pDjfXap+462n3KIo/ZktDKa+RxMFUjDZXdEAzlFQ4BeTZpuhwsAsghJ0BZxGlNCZ0n/gGzfn
+wah1ihio4mIhT8MpPnntCz9rfYNbxa8LNuExKQ/GA+h3jSH1E1w/y0zhewz5OlGNAAC4UrwV
+8y3oN4jxhqzNl4eRA3dgSlFq+tF/N2gpVDwjvmldgpcSiaVIvSFplCUbUs7vwh73qta+cJSr
+kHiNkbyLuMc3tzssvMLs+EOaJzCZlzNi0CZbiXFSp9zUctOXfqkGTcgdlK3mXspITEaJgmmU
+GL2R/xWwTiN1YHrXk3D4Etb5eZcuVm7N0a6fPHnlNuJ8JeyPei9BDXyHhtJ91wgQg/uMCmUA
+EQEAAYkCHwQYAQIACQUCUubVnwIbDAAKCRDKlsLqIWe9tDMkEADJs2/c1WxMsD/hynlSssp1
+OMidvB2aWuNZBYuBjP3ZllJYGSEZbXjwPUEb3vMUNk7xEzV/3niHEWTpEb3SpC47pSiW5aw3
+VhJIZXAT9crOILN3Bk+oJUYOPwFNxogYBnN5Rs5DoPhGtzzdb1wPwB+FXFum/+HVvXR6XotR
+AQr44PrvmC7Y5RTqJ3N8r3H+2vqcJ89FPuolxdAJ3OZ+y3L8d5ejK60+sjlPkpMYq5plbWfY
+qiyV3xzahFOH7rWVpANyvJnApp99vFSSVcev6G40U6LAcuJ1fdqlWSF0TU+iEwvdY7S5fCMO
++6s4y4+CNZJw045KP5ZniHJQmOLVGyxU9UKrjHGdmR2v/qodUCKYwONrO3vGGtyblWX70Htm
+JCzpUXzEIKUFESiyLEBzYRmuI1GV/9p/ZNX9nqWSZIrsWmVIe+Y8IAJIyP36SR5iSku0T0OB
+eyzhhD7KvN0dTl0A1XpJTr5xZXWUeig7v2WBS30D6npE/Q9axC4Y1THa2HpZWcsyIVXf3X+v
+rdKlrKaSSQzaAcggEMhsAxjGoIme1K32pNl3mhMvLdNGFTRTonw0o62HbJFEaX+E+HzbV0Ap
+o+BXfVQlGDwy1o+ZbDXQ/BcdXyj5gcjKhmqdXv3u3hrr3JnxBnbORVJ9KEq0Raxnl8NUz7GJ
+HRgYfJQv18t0mA==
+=lr4q
+-----END PGP PUBLIC KEY BLOCK-----


[11/50] [abbrv] [TWILL-35] Enabling checkstyle.

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-discovery-core/src/test/java/org/apache/twill/discovery/DiscoveryServiceTestBase.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/test/java/org/apache/twill/discovery/DiscoveryServiceTestBase.java b/twill-discovery-core/src/test/java/org/apache/twill/discovery/DiscoveryServiceTestBase.java
index 17b526b..9d86963 100644
--- a/twill-discovery-core/src/test/java/org/apache/twill/discovery/DiscoveryServiceTestBase.java
+++ b/twill-discovery-core/src/test/java/org/apache/twill/discovery/DiscoveryServiceTestBase.java
@@ -17,12 +17,12 @@
  */
 package org.apache.twill.discovery;
 
-import org.apache.twill.common.Cancellable;
-import org.apache.twill.common.Threads;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
+import org.apache.twill.common.Cancellable;
+import org.apache.twill.common.Threads;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -275,4 +275,4 @@ public abstract class DiscoveryServiceTestBase {
       throw Throwables.propagate(e);
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-discovery-core/src/test/java/org/apache/twill/discovery/ZKDiscoveryServiceTest.java
----------------------------------------------------------------------
diff --git a/twill-discovery-core/src/test/java/org/apache/twill/discovery/ZKDiscoveryServiceTest.java b/twill-discovery-core/src/test/java/org/apache/twill/discovery/ZKDiscoveryServiceTest.java
index 6f0cde0..0171b74 100644
--- a/twill-discovery-core/src/test/java/org/apache/twill/discovery/ZKDiscoveryServiceTest.java
+++ b/twill-discovery-core/src/test/java/org/apache/twill/discovery/ZKDiscoveryServiceTest.java
@@ -17,6 +17,8 @@
  */
 package org.apache.twill.discovery;
 
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.Futures;
 import org.apache.twill.common.Cancellable;
 import org.apache.twill.common.Services;
 import org.apache.twill.internal.zookeeper.InMemoryZKServer;
@@ -25,8 +27,6 @@ import org.apache.twill.zookeeper.RetryStrategies;
 import org.apache.twill.zookeeper.ZKClientService;
 import org.apache.twill.zookeeper.ZKClientServices;
 import org.apache.twill.zookeeper.ZKClients;
-import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.Futures;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
index d98dee1..e773819 100644
--- a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
+++ b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
@@ -17,11 +17,6 @@
  */
 package org.apache.twill.internal.yarn;
 
-import org.apache.twill.internal.ProcessLauncher;
-import org.apache.twill.internal.appmaster.RunnableProcessLauncher;
-import org.apache.twill.internal.yarn.ports.AMRMClient;
-import org.apache.twill.internal.yarn.ports.AMRMClientImpl;
-import org.apache.twill.internal.yarn.ports.AllocationResponse;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ArrayListMultimap;
@@ -40,6 +35,11 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.twill.internal.ProcessLauncher;
+import org.apache.twill.internal.appmaster.RunnableProcessLauncher;
+import org.apache.twill.internal.yarn.ports.AMRMClient;
+import org.apache.twill.internal.yarn.ports.AMRMClientImpl;
+import org.apache.twill.internal.yarn.ports.AllocationResponse;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
index bfec34e..6fdd99e 100644
--- a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
+++ b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAppClient.java
@@ -17,11 +17,6 @@
  */
 package org.apache.twill.internal.yarn;
 
-import org.apache.twill.api.TwillSpecification;
-import org.apache.twill.internal.ProcessController;
-import org.apache.twill.internal.ProcessLauncher;
-import org.apache.twill.internal.appmaster.ApplicationMasterProcessLauncher;
-import org.apache.twill.internal.appmaster.ApplicationSubmitter;
 import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.AbstractIdleService;
 import org.apache.hadoop.conf.Configuration;
@@ -41,6 +36,11 @@ import org.apache.hadoop.yarn.client.YarnClient;
 import org.apache.hadoop.yarn.client.YarnClientImpl;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.twill.api.TwillSpecification;
+import org.apache.twill.internal.ProcessController;
+import org.apache.twill.internal.ProcessLauncher;
+import org.apache.twill.internal.appmaster.ApplicationMasterProcessLauncher;
+import org.apache.twill.internal.appmaster.ApplicationSubmitter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnNMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnNMClient.java b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnNMClient.java
index 98ecc67..b43e4e1 100644
--- a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnNMClient.java
+++ b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnNMClient.java
@@ -17,7 +17,6 @@
  */
 package org.apache.twill.internal.yarn;
 
-import org.apache.twill.common.Cancellable;
 import com.google.common.base.Throwables;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.net.NetUtils;
@@ -32,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.twill.common.Cancellable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/ports/AllocationResponse.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/ports/AllocationResponse.java b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/ports/AllocationResponse.java
index 89734fc..d8d4c19 100644
--- a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/ports/AllocationResponse.java
+++ b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/ports/AllocationResponse.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import java.util.List;
 
 /**
- * This interface is to abstract the differences in Vanilla Hadoop YARN 2.0 and CDH 4.4
+ * This interface is to abstract the differences in Vanilla Hadoop YARN 2.0 and CDH 4.4.
  */
 public interface AllocationResponse {
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/ports/AllocationResponses.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/ports/AllocationResponses.java b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/ports/AllocationResponses.java
index ea46c3b..ba8b5fa 100644
--- a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/ports/AllocationResponses.java
+++ b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/ports/AllocationResponses.java
@@ -32,7 +32,7 @@ import java.util.List;
 public final class AllocationResponses {
 
   /**
-   * A hack for CDH 4.4.0, as the AllocateResponse class is being rewritten and diverted from YARN 2.0
+   * A hack for CDH 4.4.0, as the AllocateResponse class is being rewritten and diverted from YARN 2.0.
    */
   private static final boolean IS_CDH_4_4;
 
@@ -89,12 +89,12 @@ public final class AllocationResponses {
 
     @Override
     public List<Container> getAllocatedContainers() {
-      return call("getAllocatedContainers", new TypeToken<List<Container>>() {});
+      return call("getAllocatedContainers", new TypeToken<List<Container>>() { });
     }
 
     @Override
     public List<ContainerStatus> getCompletedContainersStatuses() {
-      return call("getCompletedContainersStatuses", new TypeToken<List<ContainerStatus>>() {});
+      return call("getCompletedContainersStatuses", new TypeToken<List<ContainerStatus>>() { });
     }
 
     private <T> T call(String methodName, TypeToken<T> resultType) {

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAMClient.java b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAMClient.java
index ce8f90f..0ebc0f5 100644
--- a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAMClient.java
+++ b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAMClient.java
@@ -17,8 +17,6 @@
  */
 package org.apache.twill.internal.yarn;
 
-import org.apache.twill.internal.ProcessLauncher;
-import org.apache.twill.internal.appmaster.RunnableProcessLauncher;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ArrayListMultimap;
@@ -38,6 +36,8 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.twill.internal.ProcessLauncher;
+import org.apache.twill.internal.appmaster.RunnableProcessLauncher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
index 50b212d..20558bb 100644
--- a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
+++ b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAppClient.java
@@ -17,11 +17,6 @@
  */
 package org.apache.twill.internal.yarn;
 
-import org.apache.twill.api.TwillSpecification;
-import org.apache.twill.internal.ProcessController;
-import org.apache.twill.internal.ProcessLauncher;
-import org.apache.twill.internal.appmaster.ApplicationMasterProcessLauncher;
-import org.apache.twill.internal.appmaster.ApplicationSubmitter;
 import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.AbstractIdleService;
 import org.apache.hadoop.conf.Configuration;
@@ -38,6 +33,11 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.client.api.YarnClientApplication;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.twill.api.TwillSpecification;
+import org.apache.twill.internal.ProcessController;
+import org.apache.twill.internal.ProcessLauncher;
+import org.apache.twill.internal.appmaster.ApplicationMasterProcessLauncher;
+import org.apache.twill.internal.appmaster.ApplicationSubmitter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnNMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnNMClient.java b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnNMClient.java
index d3a6a80..b8c9088 100644
--- a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnNMClient.java
+++ b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnNMClient.java
@@ -17,7 +17,6 @@
  */
 package org.apache.twill.internal.yarn;
 
-import org.apache.twill.common.Cancellable;
 import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.AbstractIdleService;
 import org.apache.hadoop.conf.Configuration;
@@ -26,6 +25,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.client.api.NMClient;
+import org.apache.twill.common.Cancellable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/AbstractTwillService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/AbstractTwillService.java b/twill-yarn/src/main/java/org/apache/twill/internal/AbstractTwillService.java
index 47dd07c..706039d 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/AbstractTwillService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/AbstractTwillService.java
@@ -17,14 +17,14 @@
  */
 package org.apache.twill.internal;
 
-import org.apache.twill.filesystem.Location;
-import org.apache.twill.internal.state.Message;
-import org.apache.twill.internal.state.SystemMessages;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Service;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.twill.filesystem.Location;
+import org.apache.twill.internal.state.Message;
+import org.apache.twill.internal.state.SystemMessages;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
index 4ffb023..4831158 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
@@ -17,12 +17,6 @@
  */
 package org.apache.twill.internal;
 
-import org.apache.twill.common.Services;
-import org.apache.twill.filesystem.HDFSLocationFactory;
-import org.apache.twill.filesystem.LocalLocationFactory;
-import org.apache.twill.filesystem.Location;
-import org.apache.twill.internal.logging.KafkaAppender;
-import org.apache.twill.zookeeper.ZKClientService;
 import ch.qos.logback.classic.LoggerContext;
 import ch.qos.logback.classic.joran.JoranConfigurator;
 import ch.qos.logback.classic.util.ContextInitializer;
@@ -34,6 +28,12 @@ import com.google.common.util.concurrent.Service;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.twill.common.Services;
+import org.apache.twill.filesystem.HDFSLocationFactory;
+import org.apache.twill.filesystem.LocalLocationFactory;
+import org.apache.twill.filesystem.Location;
+import org.apache.twill.internal.logging.KafkaAppender;
+import org.apache.twill.zookeeper.ZKClientService;
 import org.slf4j.ILoggerFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
index b34a7a2..880b5fd 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterMain.java
@@ -17,6 +17,10 @@
  */
 package org.apache.twill.internal.appmaster;
 
+import com.google.common.util.concurrent.Service;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.twill.api.RunId;
 import org.apache.twill.internal.Constants;
 import org.apache.twill.internal.EnvKeys;
@@ -27,10 +31,6 @@ import org.apache.twill.zookeeper.RetryStrategies;
 import org.apache.twill.zookeeper.ZKClientService;
 import org.apache.twill.zookeeper.ZKClientServices;
 import org.apache.twill.zookeeper.ZKClients;
-import com.google.common.util.concurrent.Service;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 import java.io.File;
 import java.net.InetAddress;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterProcessLauncher.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterProcessLauncher.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterProcessLauncher.java
index b51bb63..4112080 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterProcessLauncher.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterProcessLauncher.java
@@ -17,16 +17,16 @@
  */
 package org.apache.twill.internal.appmaster;
 
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.Records;
 import org.apache.twill.internal.Constants;
 import org.apache.twill.internal.EnvKeys;
 import org.apache.twill.internal.ProcessController;
 import org.apache.twill.internal.yarn.AbstractYarnProcessLauncher;
 import org.apache.twill.internal.yarn.YarnLaunchContext;
 import org.apache.twill.internal.yarn.YarnUtils;
-import com.google.common.collect.ImmutableMap;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.util.Records;
 
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
index 73cecf0..7caedad 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
@@ -566,7 +566,7 @@ public final class ApplicationMasterService extends AbstractTwillService {
       Reader reader = Files.newReader(new File(Constants.Files.LOCALIZE_FILES), Charsets.UTF_8);
       try {
         return new GsonBuilder().registerTypeAdapter(LocalFile.class, new LocalFileCodec())
-                                .create().fromJson(reader, new TypeToken<List<LocalFile>>() {}.getType());
+                                .create().fromJson(reader, new TypeToken<List<LocalFile>>() { }.getType());
       } finally {
         reader.close();
       }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationSubmitter.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationSubmitter.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationSubmitter.java
index 931c5ef..38f90ae 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationSubmitter.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationSubmitter.java
@@ -17,10 +17,10 @@
  */
 package org.apache.twill.internal.appmaster;
 
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.twill.internal.ProcessController;
 import org.apache.twill.internal.yarn.YarnApplicationReport;
 import org.apache.twill.internal.yarn.YarnLaunchContext;
-import org.apache.hadoop.yarn.api.records.Resource;
 
 /**
  * Interface for submitting a new application to run.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableContainerRequest.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableContainerRequest.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableContainerRequest.java
index 7f28443..de199ad 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableContainerRequest.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableContainerRequest.java
@@ -17,13 +17,13 @@
  */
 package org.apache.twill.internal.appmaster;
 
-import org.apache.twill.api.RuntimeSpecification;
-import org.apache.twill.api.TwillSpecification;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.twill.api.RuntimeSpecification;
+import org.apache.twill.api.TwillSpecification;
 
 import java.util.Collection;
 import java.util.Iterator;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableProcessLauncher.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableProcessLauncher.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableProcessLauncher.java
index b4b27a9..29b3f9c 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableProcessLauncher.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableProcessLauncher.java
@@ -17,6 +17,8 @@
  */
 package org.apache.twill.internal.appmaster;
 
+import com.google.common.base.Objects;
+import com.google.common.collect.Maps;
 import org.apache.twill.common.Cancellable;
 import org.apache.twill.internal.EnvKeys;
 import org.apache.twill.internal.ProcessController;
@@ -24,8 +26,6 @@ import org.apache.twill.internal.yarn.AbstractYarnProcessLauncher;
 import org.apache.twill.internal.yarn.YarnContainerInfo;
 import org.apache.twill.internal.yarn.YarnLaunchContext;
 import org.apache.twill.internal.yarn.YarnNMClient;
-import com.google.common.base.Objects;
-import com.google.common.collect.Maps;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
index beef0d4..57c58da 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
@@ -17,19 +17,6 @@
  */
 package org.apache.twill.internal.appmaster;
 
-import org.apache.twill.api.ResourceReport;
-import org.apache.twill.api.RunId;
-import org.apache.twill.api.ServiceController;
-import org.apache.twill.api.TwillRunResources;
-import org.apache.twill.internal.ContainerInfo;
-import org.apache.twill.internal.DefaultResourceReport;
-import org.apache.twill.internal.DefaultTwillRunResources;
-import org.apache.twill.internal.RunIds;
-import org.apache.twill.internal.TwillContainerController;
-import org.apache.twill.internal.TwillContainerLauncher;
-import org.apache.twill.internal.container.TwillContainerMain;
-import org.apache.twill.internal.state.Message;
-import org.apache.twill.internal.yarn.YarnContainerStatus;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.HashBasedTable;
@@ -43,6 +30,19 @@ import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.twill.api.ResourceReport;
+import org.apache.twill.api.RunId;
+import org.apache.twill.api.ServiceController;
+import org.apache.twill.api.TwillRunResources;
+import org.apache.twill.internal.ContainerInfo;
+import org.apache.twill.internal.DefaultResourceReport;
+import org.apache.twill.internal.DefaultTwillRunResources;
+import org.apache.twill.internal.RunIds;
+import org.apache.twill.internal.TwillContainerController;
+import org.apache.twill.internal.TwillContainerLauncher;
+import org.apache.twill.internal.container.TwillContainerMain;
+import org.apache.twill.internal.state.Message;
+import org.apache.twill.internal.yarn.YarnContainerStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -334,7 +334,7 @@ final class RunningContainers {
   }
 
   /**
-   * Sends a command through the given {@link org.apache.twill.internal.TwillContainerController} of a runnable. Decrements the count
+   * Sends a command through the given {@link TwillContainerController} of a runnable. Decrements the count
    * when the sending of command completed. Triggers completion when count reaches zero.
    */
   private void sendMessage(final String runnableName, final Message message,

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
index ca299e0..8ff5184 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
@@ -17,10 +17,10 @@
  */
 package org.apache.twill.internal.appmaster;
 
-import org.apache.twill.api.ResourceReport;
-import org.apache.twill.internal.json.ResourceReportAdapter;
 import com.google.common.util.concurrent.AbstractIdleService;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.twill.api.ResourceReport;
+import org.apache.twill.internal.json.ResourceReportAdapter;
 import org.jboss.netty.bootstrap.ServerBootstrap;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBufferOutputStream;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
index bbd6c10..072bbb0 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
@@ -17,6 +17,15 @@
  */
 package org.apache.twill.internal.container;
 
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.io.Files;
+import com.google.common.util.concurrent.Service;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.twill.api.LocalFile;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.RuntimeSpecification;
@@ -39,15 +48,6 @@ import org.apache.twill.zookeeper.ZKClient;
 import org.apache.twill.zookeeper.ZKClientService;
 import org.apache.twill.zookeeper.ZKClientServices;
 import org.apache.twill.zookeeper.ZKClients;
-import com.google.common.base.Charsets;
-import com.google.common.base.Preconditions;
-import com.google.common.io.Files;
-import com.google.common.util.concurrent.Service;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
index f5bc1f2..9890f17 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
@@ -17,6 +17,15 @@
  */
 package org.apache.twill.internal.container;
 
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.util.concurrent.AbstractExecutionThreadService;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.Service;
+import com.google.common.util.concurrent.SettableFuture;
+import com.google.gson.Gson;
+import com.google.gson.JsonElement;
 import org.apache.twill.api.Command;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.TwillRunnable;
@@ -24,7 +33,6 @@ import org.apache.twill.api.TwillRunnableSpecification;
 import org.apache.twill.common.Threads;
 import org.apache.twill.filesystem.Location;
 import org.apache.twill.internal.AbstractTwillService;
-import org.apache.twill.internal.AbstractTwillService;
 import org.apache.twill.internal.BasicTwillContext;
 import org.apache.twill.internal.ContainerInfo;
 import org.apache.twill.internal.ContainerLiveNodeData;
@@ -34,15 +42,6 @@ import org.apache.twill.internal.state.Message;
 import org.apache.twill.internal.state.MessageCallback;
 import org.apache.twill.internal.utils.Instances;
 import org.apache.twill.zookeeper.ZKClient;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
-import com.google.common.util.concurrent.AbstractExecutionThreadService;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.Service;
-import com.google.common.util.concurrent.SettableFuture;
-import com.google.gson.Gson;
-import com.google.gson.JsonElement;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnProcessLauncher.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnProcessLauncher.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnProcessLauncher.java
index b810854..1c28c47 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnProcessLauncher.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnProcessLauncher.java
@@ -17,16 +17,16 @@
  */
 package org.apache.twill.internal.yarn;
 
-import org.apache.twill.api.LocalFile;
-import org.apache.twill.internal.ProcessController;
-import org.apache.twill.internal.ProcessLauncher;
-import org.apache.twill.internal.utils.Paths;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.twill.api.LocalFile;
+import org.apache.twill.internal.ProcessController;
+import org.apache.twill.internal.ProcessLauncher;
+import org.apache.twill.internal.utils.Paths;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
index 83ba6a8..b0dbce0 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
@@ -17,13 +17,13 @@
  */
 package org.apache.twill.internal.yarn;
 
-import org.apache.twill.internal.ProcessLauncher;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.Service;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.twill.internal.ProcessLauncher;
 
 import java.net.InetSocketAddress;
 import java.net.URL;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAppClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAppClient.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAppClient.java
index 71a9e68..97236f6 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAppClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAppClient.java
@@ -17,11 +17,11 @@
  */
 package org.apache.twill.internal.yarn;
 
+import com.google.common.util.concurrent.Service;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.twill.api.TwillSpecification;
 import org.apache.twill.internal.ProcessController;
 import org.apache.twill.internal.ProcessLauncher;
-import com.google.common.util.concurrent.Service;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 
 /**
  * Interface for launching Yarn application from client.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnApplicationReport.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnApplicationReport.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnApplicationReport.java
index 4dbb1d1..c3382d5 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnApplicationReport.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnApplicationReport.java
@@ -36,7 +36,7 @@ public interface YarnApplicationReport {
 
   /**
    * Get the <code>ApplicationAttemptId</code> of the current
-   * attempt of the application
+   * attempt of the application.
    * @return <code>ApplicationAttemptId</code> of the attempt
    */
   ApplicationAttemptId getCurrentApplicationAttemptId();
@@ -119,7 +119,7 @@ public interface YarnApplicationReport {
   FinalApplicationStatus getFinalApplicationStatus();
 
   /**
-   * Retrieve the structure containing the job resources for this application
+   * Retrieve the structure containing the job resources for this application.
    * @return the job resources structure for this application
    */
   ApplicationResourceUsageReport getApplicationResourceUsageReport();

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnContainerStatus.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnContainerStatus.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnContainerStatus.java
index 57e712c..3ca630a 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnContainerStatus.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnContainerStatus.java
@@ -20,7 +20,7 @@ package org.apache.twill.internal.yarn;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 
 /**
- * This interface is for adapting differences in ContainerStatus between Hadoop 2.0 and 2.1
+ * This interface is for adapting differences in ContainerStatus between Hadoop 2.0 and 2.1.
  */
 public interface YarnContainerStatus {
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnLaunchContext.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnLaunchContext.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnLaunchContext.java
index 984a1be..390a633 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnLaunchContext.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnLaunchContext.java
@@ -25,7 +25,7 @@ import java.util.List;
 import java.util.Map;
 
 /**
- * This interface is for adapting ContainerLaunchContext in different Hadoop version
+ * This interface is for adapting ContainerLaunchContext in different Hadoop version.
  */
 public interface YarnLaunchContext {
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnNMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnNMClient.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnNMClient.java
index d863c91..11c73b9 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnNMClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnNMClient.java
@@ -20,7 +20,7 @@ package org.apache.twill.internal.yarn;
 import org.apache.twill.common.Cancellable;
 
 /**
- * Abstraction for dealing with API differences in different hadoop yarn version
+ * Abstraction for dealing with API differences in different hadoop yarn version.
  */
 public interface YarnNMClient {
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
index 4f7597b..be30f33 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnUtils.java
@@ -17,10 +17,6 @@
  */
 package org.apache.twill.internal.yarn;
 
-import org.apache.twill.api.LocalFile;
-import org.apache.twill.filesystem.ForwardingLocationFactory;
-import org.apache.twill.filesystem.HDFSLocationFactory;
-import org.apache.twill.filesystem.LocationFactory;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
@@ -41,6 +37,10 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.twill.api.LocalFile;
+import org.apache.twill.filesystem.ForwardingLocationFactory;
+import org.apache.twill.filesystem.HDFSLocationFactory;
+import org.apache.twill.filesystem.LocationFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -220,7 +220,7 @@ public class YarnUtils {
   }
 
   /**
-   * Helper method to create adapter class for bridging between Hadoop 2.0 and 2.1
+   * Helper method to create adapter class for bridging between Hadoop 2.0 and 2.1.
    */
   private static <T> T createAdapter(Class<T> clz) {
     String className = clz.getPackage().getName();

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreUpdater.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreUpdater.java b/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreUpdater.java
index 4d20c9c..02fd356 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreUpdater.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/LocationSecureStoreUpdater.java
@@ -17,14 +17,14 @@
  */
 package org.apache.twill.yarn;
 
+import com.google.common.base.Throwables;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Credentials;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.SecureStore;
 import org.apache.twill.api.SecureStoreUpdater;
 import org.apache.twill.filesystem.LocationFactory;
 import org.apache.twill.internal.yarn.YarnUtils;
-import com.google.common.base.Throwables;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.Credentials;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/yarn/ResourceReportClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/ResourceReportClient.java b/twill-yarn/src/main/java/org/apache/twill/yarn/ResourceReportClient.java
index 2974c3f..3d5bcf3 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/ResourceReportClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/ResourceReportClient.java
@@ -17,10 +17,10 @@
  */
 package org.apache.twill.yarn;
 
-import org.apache.twill.api.ResourceReport;
-import org.apache.twill.internal.json.ResourceReportAdapter;
 import com.google.common.base.Charsets;
 import com.google.common.io.Closeables;
+import org.apache.twill.api.ResourceReport;
+import org.apache.twill.internal.json.ResourceReportAdapter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/yarn/YarnSecureStore.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnSecureStore.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnSecureStore.java
index e6f461a..77c736f 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnSecureStore.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnSecureStore.java
@@ -17,8 +17,8 @@
  */
 package org.apache.twill.yarn;
 
-import org.apache.twill.api.SecureStore;
 import org.apache.hadoop.security.Credentials;
+import org.apache.twill.api.SecureStore;
 
 /**
  * A {@link SecureStore} for hadoop credentials.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
index 4c240fb..8e0c6be 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
@@ -17,6 +17,13 @@
  */
 package org.apache.twill.yarn;
 
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang.time.StopWatch;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.twill.api.ResourceReport;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.TwillController;
@@ -30,13 +37,6 @@ import org.apache.twill.internal.state.SystemMessages;
 import org.apache.twill.internal.yarn.YarnApplicationReport;
 import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.ZKClient;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.Uninterruptibles;
-import org.apache.commons.lang.time.StopWatch;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
index baa5740..9dff24d 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
@@ -17,6 +17,36 @@
  */
 package org.apache.twill.yarn;
 
+import com.google.common.base.Charsets;
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.base.Suppliers;
+import com.google.common.base.Throwables;
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.ImmutableTable;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
+import com.google.common.collect.Table;
+import com.google.common.util.concurrent.AbstractIdleService;
+import com.google.common.util.concurrent.Callables;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.gson.Gson;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.SecureStore;
@@ -51,36 +81,6 @@ import org.apache.twill.zookeeper.ZKClientService;
 import org.apache.twill.zookeeper.ZKClientServices;
 import org.apache.twill.zookeeper.ZKClients;
 import org.apache.twill.zookeeper.ZKOperations;
-import com.google.common.base.Charsets;
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import com.google.common.base.Suppliers;
-import com.google.common.base.Throwables;
-import com.google.common.collect.HashBasedTable;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.ImmutableTable;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Sets;
-import com.google.common.collect.Table;
-import com.google.common.util.concurrent.AbstractIdleService;
-import com.google.common.util.concurrent.Callables;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.gson.Gson;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonObject;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/test/java/org/apache/twill/yarn/BaseYarnTest.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/BaseYarnTest.java b/twill-yarn/src/test/java/org/apache/twill/yarn/BaseYarnTest.java
index 6943ef9..14b7f67 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/BaseYarnTest.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/BaseYarnTest.java
@@ -17,12 +17,12 @@
  */
 package org.apache.twill.yarn;
 
-import java.io.IOException;
-
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.rules.TemporaryFolder;
 
+import java.io.IOException;
+
 /**
  * Base class for all YARN tests.
  */

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/test/java/org/apache/twill/yarn/DistributeShellTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/DistributeShellTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/DistributeShellTestRun.java
index 0ed496e..10cc09f 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/DistributeShellTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/DistributeShellTestRun.java
@@ -17,21 +17,19 @@
  */
 package org.apache.twill.yarn;
 
-import java.io.PrintWriter;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
-
 import com.google.common.util.concurrent.Service;
-
 import org.apache.twill.api.TwillController;
 import org.apache.twill.api.TwillRunner;
 import org.apache.twill.api.logging.PrinterLogHandler;
 import org.apache.twill.common.ServiceListenerAdapter;
 import org.apache.twill.common.Threads;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.io.PrintWriter;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 /**
  * This test is executed by {@link YarnTestSuite}.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/test/java/org/apache/twill/yarn/DistributedShell.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/DistributedShell.java b/twill-yarn/src/test/java/org/apache/twill/yarn/DistributedShell.java
index 2f42e31..fb37820 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/DistributedShell.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/DistributedShell.java
@@ -17,20 +17,18 @@
  */
 package org.apache.twill.yarn;
 
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 import com.google.common.base.Splitter;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-
+import org.apache.twill.api.AbstractTwillRunnable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.twill.api.AbstractTwillRunnable;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
index a14b326..23fc82b 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
@@ -17,25 +17,8 @@
  */
 package org.apache.twill.yarn;
 
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
-import java.net.Socket;
-import java.net.URISyntaxException;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
 import com.google.common.base.Charsets;
 import com.google.common.io.LineReader;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.api.TwillController;
 import org.apache.twill.api.TwillRunner;
@@ -44,6 +27,21 @@ import org.apache.twill.api.logging.PrinterLogHandler;
 import org.apache.twill.common.ServiceListenerAdapter;
 import org.apache.twill.common.Threads;
 import org.apache.twill.discovery.Discoverable;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.net.Socket;
+import java.net.URISyntaxException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 /**
  * Using echo server to test various behavior of YarnTwillService.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/test/java/org/apache/twill/yarn/FailureRestartTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/FailureRestartTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/FailureRestartTestRun.java
index 8bc9571..56f1182 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/FailureRestartTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/FailureRestartTestRun.java
@@ -102,6 +102,9 @@ public final class FailureRestartTestRun extends BaseYarnTest {
   }
 
 
+  /**
+   * A SocketServer that fails upon receiving a kill command.
+   */
   public static final class FailureRunnable extends SocketServer {
 
     private volatile boolean killed;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
index d7e186f..78f61a9 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
@@ -17,30 +17,11 @@
  */
 package org.apache.twill.yarn;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.concurrent.TimeUnit;
-import java.util.jar.JarEntry;
-import java.util.jar.JarOutputStream;
-
-import org.junit.Test;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.io.ByteStreams;
 import com.google.common.io.Files;
 import com.google.common.io.LineReader;
-
 import org.apache.twill.api.TwillApplication;
 import org.apache.twill.api.TwillController;
 import org.apache.twill.api.TwillRunner;
@@ -48,6 +29,22 @@ import org.apache.twill.api.TwillSpecification;
 import org.apache.twill.api.logging.PrinterLogHandler;
 import org.apache.twill.discovery.Discoverable;
 import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.concurrent.TimeUnit;
+import java.util.jar.JarEntry;
+import java.util.jar.JarOutputStream;
 
 /**
  * Test for local file transfer.
@@ -101,6 +98,9 @@ public final class LocalFileTestRun extends BaseYarnTest {
     TimeUnit.SECONDS.sleep(2);
   }
 
+  /**
+   * Application for testing local file transfer.
+   */
   public static final class LocalFileApplication implements TwillApplication {
 
     private final File headerFile;
@@ -130,6 +130,9 @@ public final class LocalFileTestRun extends BaseYarnTest {
     }
   }
 
+  /**
+   * SocketServer for testing local file transfer.
+   */
   public static final class LocalFileSocketServer extends SocketServer {
 
     private static final Logger LOG = LoggerFactory.getLogger(LocalFileSocketServer.class);

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/test/java/org/apache/twill/yarn/ProvisionTimeoutTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/ProvisionTimeoutTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/ProvisionTimeoutTestRun.java
index d5e3fc3..20acb18 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/ProvisionTimeoutTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/ProvisionTimeoutTestRun.java
@@ -17,16 +17,8 @@
  */
 package org.apache.twill.yarn;
 
-import java.io.PrintWriter;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableMap;
-
 import org.apache.twill.api.AbstractTwillRunnable;
 import org.apache.twill.api.EventHandler;
 import org.apache.twill.api.EventHandlerContext;
@@ -35,10 +27,16 @@ import org.apache.twill.api.TwillApplication;
 import org.apache.twill.api.TwillController;
 import org.apache.twill.api.TwillRunner;
 import org.apache.twill.api.TwillSpecification;
-import org.junit.Test;
-
 import org.apache.twill.api.logging.PrinterLogHandler;
 import org.apache.twill.common.Services;
+import org.junit.Test;
+
+import java.io.PrintWriter;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 /**
  *
@@ -63,6 +61,9 @@ public final class ProvisionTimeoutTestRun extends BaseYarnTest {
     }
   }
 
+  /**
+   * The handler for testing timeout handling.
+   */
   public static final class Handler extends EventHandler {
 
     private boolean abort;
@@ -87,6 +88,9 @@ public final class ProvisionTimeoutTestRun extends BaseYarnTest {
     }
   }
 
+  /**
+   * Testing application for timeout.
+   */
   public static final class TimeoutApplication implements TwillApplication {
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/test/java/org/apache/twill/yarn/SocketServer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/SocketServer.java b/twill-yarn/src/test/java/org/apache/twill/yarn/SocketServer.java
index dabdc07..a17704a 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/SocketServer.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/SocketServer.java
@@ -128,5 +128,5 @@ public abstract class SocketServer extends AbstractTwillRunnable {
     }
   }
 
-  abstract public void handleRequest(BufferedReader reader, PrintWriter writer) throws IOException;
+  public abstract void handleRequest(BufferedReader reader, PrintWriter writer) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-yarn/src/test/java/org/apache/twill/yarn/TaskCompletedTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/TaskCompletedTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/TaskCompletedTestRun.java
index 37fc8aa..420b249 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/TaskCompletedTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/TaskCompletedTestRun.java
@@ -40,6 +40,9 @@ import java.util.concurrent.TimeUnit;
  */
 public final class TaskCompletedTestRun extends BaseYarnTest {
 
+  /**
+   * A {@link AbstractTwillRunnable} that sleeps randomly and finish.
+   */
   public static final class SleepTask extends AbstractTwillRunnable {
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicNodeChildren.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicNodeChildren.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicNodeChildren.java
index 9e4f55f..cad233e 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicNodeChildren.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicNodeChildren.java
@@ -17,8 +17,8 @@
  */
 package org.apache.twill.internal.zookeeper;
 
-import org.apache.twill.zookeeper.NodeChildren;
 import com.google.common.base.Objects;
+import org.apache.twill.zookeeper.NodeChildren;
 import org.apache.zookeeper.data.Stat;
 
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicNodeData.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicNodeData.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicNodeData.java
index 98a3a66..13df625 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicNodeData.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicNodeData.java
@@ -17,8 +17,8 @@
  */
 package org.apache.twill.internal.zookeeper;
 
-import org.apache.twill.zookeeper.NodeData;
 import com.google.common.base.Objects;
+import org.apache.twill.zookeeper.NodeData;
 import org.apache.zookeeper.data.Stat;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/DefaultZKClientService.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/DefaultZKClientService.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/DefaultZKClientService.java
index c52fb08..84b3a8d 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/DefaultZKClientService.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/DefaultZKClientService.java
@@ -17,11 +17,6 @@
  */
 package org.apache.twill.internal.zookeeper;
 
-import org.apache.twill.common.Threads;
-import org.apache.twill.zookeeper.NodeChildren;
-import org.apache.twill.zookeeper.NodeData;
-import org.apache.twill.zookeeper.OperationFuture;
-import org.apache.twill.zookeeper.ZKClientService;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
@@ -30,6 +25,11 @@ import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Service;
+import org.apache.twill.common.Threads;
+import org.apache.twill.zookeeper.NodeChildren;
+import org.apache.twill.zookeeper.NodeData;
+import org.apache.twill.zookeeper.OperationFuture;
+import org.apache.twill.zookeeper.ZKClientService;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -42,7 +42,6 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.CopyOnWriteArrayList;
@@ -52,6 +51,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
+import javax.annotation.Nullable;
 
 /**
  * The base implementation of {@link ZKClientService}.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/FailureRetryZKClient.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/FailureRetryZKClient.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/FailureRetryZKClient.java
index 65ceadb..aa11730 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/FailureRetryZKClient.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/FailureRetryZKClient.java
@@ -17,6 +17,9 @@
  */
 package org.apache.twill.internal.zookeeper;
 
+import com.google.common.base.Supplier;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
 import org.apache.twill.common.Threads;
 import org.apache.twill.zookeeper.ForwardingZKClient;
 import org.apache.twill.zookeeper.NodeChildren;
@@ -25,9 +28,6 @@ import org.apache.twill.zookeeper.OperationFuture;
 import org.apache.twill.zookeeper.RetryStrategy;
 import org.apache.twill.zookeeper.RetryStrategy.OperationType;
 import org.apache.twill.zookeeper.ZKClient;
-import com.google.common.base.Supplier;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/NamespaceZKClient.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/NamespaceZKClient.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/NamespaceZKClient.java
index 1a82e4b..7d3c268 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/NamespaceZKClient.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/NamespaceZKClient.java
@@ -17,14 +17,14 @@
  */
 package org.apache.twill.internal.zookeeper;
 
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
 import org.apache.twill.common.Threads;
 import org.apache.twill.zookeeper.ForwardingZKClient;
 import org.apache.twill.zookeeper.NodeChildren;
 import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.OperationFuture;
 import org.apache.twill.zookeeper.ZKClient;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireWatcher.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireWatcher.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireWatcher.java
index 181ca2b..776efe4 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireWatcher.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireWatcher.java
@@ -17,11 +17,11 @@
  */
 package org.apache.twill.internal.zookeeper;
 
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
 import org.apache.twill.zookeeper.NodeChildren;
 import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.ZKClient;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
index 402c153..70db61c 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
@@ -17,14 +17,14 @@
  */
 package org.apache.twill.internal.zookeeper;
 
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
 import org.apache.twill.internal.zookeeper.RewatchOnExpireWatcher.ActionType;
 import org.apache.twill.zookeeper.ForwardingZKClient;
 import org.apache.twill.zookeeper.NodeChildren;
 import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.OperationFuture;
 import org.apache.twill.zookeeper.ZKClient;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/SettableOperationFuture.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/SettableOperationFuture.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/SettableOperationFuture.java
index 7544e56..06f089e 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/SettableOperationFuture.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/SettableOperationFuture.java
@@ -17,15 +17,17 @@
  */
 package org.apache.twill.internal.zookeeper;
 
-import org.apache.twill.zookeeper.OperationFuture;
 import com.google.common.util.concurrent.AbstractFuture;
+import org.apache.twill.zookeeper.OperationFuture;
 
-import javax.annotation.Nullable;
 import java.util.concurrent.Executor;
+import javax.annotation.Nullable;
 
 /**
  * An implementation for {@link OperationFuture} that allows setting result directly.
  * Also, all listener callback will be fired from the given executor.
+ *
+ * @param <V> The result type returned by this Future's {@link #get()} method.
  */
 public final class SettableOperationFuture<V> extends AbstractFuture<V> implements OperationFuture<V> {
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientService.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientService.java b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientService.java
index 63f27fb..8d159af 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientService.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientService.java
@@ -17,11 +17,11 @@
  */
 package org.apache.twill.zookeeper;
 
-import org.apache.twill.internal.zookeeper.DefaultZKClientService;
 import com.google.common.base.Supplier;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 import com.google.common.util.concurrent.Service;
+import org.apache.twill.internal.zookeeper.DefaultZKClientService;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.ACL;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKOperations.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKOperations.java b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKOperations.java
index 6dcd1a7..0e2239d 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKOperations.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKOperations.java
@@ -17,14 +17,14 @@
  */
 package org.apache.twill.zookeeper;
 
-import org.apache.twill.common.Cancellable;
-import org.apache.twill.common.Threads;
-import org.apache.twill.internal.zookeeper.SettableOperationFuture;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
+import org.apache.twill.common.Cancellable;
+import org.apache.twill.common.Threads;
+import org.apache.twill.internal.zookeeper.SettableOperationFuture;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/098a0cac/twill-zookeeper/src/test/java/org/apache/twill/zookeeper/ZKClientTest.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/test/java/org/apache/twill/zookeeper/ZKClientTest.java b/twill-zookeeper/src/test/java/org/apache/twill/zookeeper/ZKClientTest.java
index f1db74a..2228d46 100644
--- a/twill-zookeeper/src/test/java/org/apache/twill/zookeeper/ZKClientTest.java
+++ b/twill-zookeeper/src/test/java/org/apache/twill/zookeeper/ZKClientTest.java
@@ -17,14 +17,14 @@
  */
 package org.apache.twill.zookeeper;
 
-import org.apache.twill.internal.zookeeper.InMemoryZKServer;
-import org.apache.twill.internal.zookeeper.KillZKSession;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.io.Files;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import org.apache.twill.internal.zookeeper.InMemoryZKServer;
+import org.apache.twill.internal.zookeeper.KillZKSession;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;


[48/50] [abbrv] git commit: Workaround for YARN-314 by not mixing new container request with remove container request (request with container count = 0).

Posted by ch...@apache.org.
Workaround for YARN-314 by not mixing new container request with remove container request (request with container count = 0).

Signed-off-by: Terence Yim <ch...@apache.org>


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

Branch: refs/heads/site
Commit: 2c3cf39682415420a89f0360c0abe9a42fdc4abe
Parents: d6504eb
Author: Terence Yim <te...@continuuity.com>
Authored: Wed Apr 16 17:06:25 2014 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Thu Apr 17 00:26:25 2014 -0700

----------------------------------------------------------------------
 .../internal/yarn/Hadoop20YarnAMClient.java     | 139 ++++-------
 .../internal/yarn/Hadoop21YarnAMClient.java     | 100 ++------
 .../appmaster/ApplicationMasterService.java     |   6 +-
 .../internal/yarn/AbstractYarnAMClient.java     | 229 +++++++++++++++++++
 .../twill/internal/yarn/YarnAMClient.java       |   4 +-
 .../apache/twill/yarn/ContainerSizeTestRun.java | 119 ++++++++++
 .../apache/twill/yarn/LogHandlerTestRun.java    |   3 +
 7 files changed, 420 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/2c3cf396/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
index 68d073d..9b66f67 100644
--- a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
+++ b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
@@ -19,23 +19,18 @@ package org.apache.twill.internal.yarn;
 
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
-import com.google.common.util.concurrent.AbstractIdleService;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.twill.internal.ProcessLauncher;
 import org.apache.twill.internal.appmaster.RunnableProcessLauncher;
 import org.apache.twill.internal.yarn.ports.AMRMClient;
 import org.apache.twill.internal.yarn.ports.AMRMClientImpl;
@@ -43,15 +38,13 @@ import org.apache.twill.internal.yarn.ports.AllocationResponse;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.InetSocketAddress;
-import java.net.URL;
 import java.util.List;
-import java.util.UUID;
+import javax.annotation.Nullable;
 
 /**
  *
  */
-public final class Hadoop20YarnAMClient extends AbstractIdleService implements YarnAMClient {
+public final class Hadoop20YarnAMClient extends AbstractYarnAMClient<AMRMClient.ContainerRequest> {
 
   private static final Logger LOG = LoggerFactory.getLogger(Hadoop20YarnAMClient.class);
   private static final Function<ContainerStatus, YarnContainerStatus> STATUS_TRANSFORM;
@@ -65,21 +58,13 @@ public final class Hadoop20YarnAMClient extends AbstractIdleService implements Y
     };
   }
 
-  private final ContainerId containerId;
-  private final Multimap<String, AMRMClient.ContainerRequest> containerRequests;
   private final AMRMClient amrmClient;
   private final YarnNMClient nmClient;
-  private InetSocketAddress trackerAddr;
-  private URL trackerUrl;
   private Resource maxCapability;
   private Resource minCapability;
 
   public Hadoop20YarnAMClient(Configuration conf) {
-    String masterContainerId = System.getenv().get(ApplicationConstants.AM_CONTAINER_ID_ENV);
-    Preconditions.checkArgument(masterContainerId != null,
-                                "Missing %s from environment", ApplicationConstants.AM_CONTAINER_ID_ENV);
-    this.containerId = ConverterUtils.toContainerId(masterContainerId);
-    this.containerRequests = ArrayListMultimap.create();
+    super(ApplicationConstants.AM_CONTAINER_ID_ENV);
 
     this.amrmClient = new AMRMClientImpl(containerId.getApplicationAttemptId());
     this.amrmClient.init(conf);
@@ -110,107 +95,67 @@ public final class Hadoop20YarnAMClient extends AbstractIdleService implements Y
   }
 
   @Override
-  public ContainerId getContainerId() {
-    return containerId;
-  }
-
-  @Override
   public String getHost() {
     return System.getenv().get(ApplicationConstants.NM_HOST_ENV);
   }
 
   @Override
-  public void setTracker(InetSocketAddress trackerAddr, URL trackerUrl) {
-    this.trackerAddr = trackerAddr;
-    this.trackerUrl = trackerUrl;
-  }
-
-  @Override
-  public synchronized void allocate(float progress, AllocateHandler handler) throws Exception {
-    AllocationResponse response = amrmClient.allocate(progress);
-    List<ProcessLauncher<YarnContainerInfo>> launchers
-      = Lists.newArrayListWithCapacity(response.getAllocatedContainers().size());
-
-    for (Container container : response.getAllocatedContainers()) {
-      launchers.add(new RunnableProcessLauncher(new Hadoop20YarnContainerInfo(container), nmClient));
+  protected Resource adjustCapability(Resource resource) {
+    int cores = YarnUtils.getVirtualCores(resource);
+    int updatedCores = Math.max(Math.min(cores, YarnUtils.getVirtualCores(maxCapability)),
+                                YarnUtils.getVirtualCores(minCapability));
+    // Try and set the virtual cores, which older versions of YARN don't support this.
+    if (cores != updatedCores && YarnUtils.setVirtualCores(resource, updatedCores)) {
+      LOG.info("Adjust virtual cores requirement from {} to {}.", cores, updatedCores);
     }
 
-    if (!launchers.isEmpty()) {
-      handler.acquired(launchers);
-
-      // If no process has been launched through the given launcher, return the container.
-      for (ProcessLauncher<YarnContainerInfo> l : launchers) {
-        // This cast always works.
-        RunnableProcessLauncher launcher = (RunnableProcessLauncher) l;
-        if (!launcher.isLaunched()) {
-          Container container = launcher.getContainerInfo().getContainer();
-          LOG.info("Nothing to run in container, releasing it: {}", container);
-          amrmClient.releaseAssignedContainer(container.getId());
-        }
-      }
-    }
+    int updatedMemory = Math.min(resource.getMemory(), maxCapability.getMemory());
+    int minMemory = minCapability.getMemory();
+    updatedMemory = (int) Math.ceil(((double) updatedMemory / minMemory)) * minMemory;
 
-    List<YarnContainerStatus> completed = ImmutableList.copyOf(
-      Iterables.transform(response.getCompletedContainersStatuses(), STATUS_TRANSFORM));
-    if (!completed.isEmpty()) {
-      handler.completed(completed);
+    if (resource.getMemory() != updatedMemory) {
+      resource.setMemory(updatedMemory);
+      LOG.info("Adjust memory requirement from {} to {} MB.", resource.getMemory(), updatedMemory);
     }
+
+    return resource;
   }
 
   @Override
-  public ContainerRequestBuilder addContainerRequest(Resource capability) {
-    return addContainerRequest(capability, 1);
+  protected AMRMClient.ContainerRequest createContainerRequest(Priority priority, Resource capability,
+                                                               @Nullable String[] hosts, @Nullable String[] racks) {
+    return new AMRMClient.ContainerRequest(capability, hosts, racks, priority, 1);
   }
 
   @Override
-  public ContainerRequestBuilder addContainerRequest(Resource capability, int count) {
-    return new ContainerRequestBuilder(adjustCapability(capability), count) {
-      @Override
-      public String apply() {
-        synchronized (Hadoop20YarnAMClient.this) {
-          String id = UUID.randomUUID().toString();
-
-          String[] hosts = this.hosts.isEmpty() ? null : this.hosts.toArray(new String[this.hosts.size()]);
-          String[] racks = this.racks.isEmpty() ? null : this.racks.toArray(new String[this.racks.size()]);
-
-          for (int i = 0; i < count; i++) {
-            AMRMClient.ContainerRequest request = new AMRMClient.ContainerRequest(capability, hosts, racks,
-                                                                                  priority, 1);
-            containerRequests.put(id, request);
-            amrmClient.addContainerRequest(request);
-          }
-
-          return id;
-        }
-      }
-    };
+  protected void addContainerRequest(AMRMClient.ContainerRequest request) {
+    amrmClient.addContainerRequest(request);
   }
 
   @Override
-  public synchronized void completeContainerRequest(String id) {
-    for (AMRMClient.ContainerRequest request : containerRequests.removeAll(id)) {
-      amrmClient.removeContainerRequest(request);
-    }
+  protected void removeContainerRequest(AMRMClient.ContainerRequest request) {
+    amrmClient.removeContainerRequest(request);
   }
 
-  private Resource adjustCapability(Resource resource) {
-    int cores = YarnUtils.getVirtualCores(resource);
-    int updatedCores = Math.max(Math.min(cores, YarnUtils.getVirtualCores(maxCapability)),
-                                YarnUtils.getVirtualCores(minCapability));
-    // Try and set the virtual cores, which older versions of YARN don't support this.
-    if (cores != updatedCores && YarnUtils.setVirtualCores(resource, updatedCores)) {
-      LOG.info("Adjust virtual cores requirement from {} to {}.", cores, updatedCores);
+  @Override
+  protected AllocateResult doAllocate(float progress) throws Exception {
+    AllocationResponse response = amrmClient.allocate(progress);
+    List<RunnableProcessLauncher> launchers
+      = Lists.newArrayListWithCapacity(response.getAllocatedContainers().size());
+
+    for (Container container : response.getAllocatedContainers()) {
+      launchers.add(new RunnableProcessLauncher(new Hadoop20YarnContainerInfo(container), nmClient));
     }
 
-    int updatedMemory = Math.min(resource.getMemory(), maxCapability.getMemory());
-    int minMemory = minCapability.getMemory();
-    updatedMemory = (int) Math.ceil(((double) updatedMemory / minMemory)) * minMemory;
+    List<YarnContainerStatus> completed = ImmutableList.copyOf(
+      Iterables.transform(response.getCompletedContainersStatuses(), STATUS_TRANSFORM));
 
-    if (resource.getMemory() != updatedMemory) {
-      resource.setMemory(updatedMemory);
-      LOG.info("Adjust memory requirement from {} to {} MB.", resource.getMemory(), updatedMemory);
-    }
+    return new AllocateResult(launchers, completed);
+  }
 
-    return resource;
+  @Override
+  protected void releaseAssignedContainer(YarnContainerInfo containerInfo) {
+    Container container = containerInfo.getContainer();
+    amrmClient.releaseAssignedContainer(container.getId());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/2c3cf396/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAMClient.java b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAMClient.java
index 0ebc0f5..78a5135 100644
--- a/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAMClient.java
+++ b/twill-yarn/src/main/hadoop21/org/apache/twill/internal/yarn/Hadoop21YarnAMClient.java
@@ -19,37 +19,30 @@ package org.apache.twill.internal.yarn;
 
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
-import com.google.common.util.concurrent.AbstractIdleService;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.twill.internal.ProcessLauncher;
 import org.apache.twill.internal.appmaster.RunnableProcessLauncher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.InetSocketAddress;
-import java.net.URL;
 import java.util.List;
-import java.util.UUID;
+import javax.annotation.Nullable;
 
 /**
  *
  */
-public final class Hadoop21YarnAMClient extends AbstractIdleService implements YarnAMClient {
+public final class Hadoop21YarnAMClient extends AbstractYarnAMClient<AMRMClient.ContainerRequest> {
 
   private static final Logger LOG = LoggerFactory.getLogger(Hadoop21YarnAMClient.class);
 
@@ -64,20 +57,12 @@ public final class Hadoop21YarnAMClient extends AbstractIdleService implements Y
     };
   }
 
-  private final ContainerId containerId;
-  private final Multimap<String, AMRMClient.ContainerRequest> containerRequests;
   private final AMRMClient<AMRMClient.ContainerRequest> amrmClient;
   private final Hadoop21YarnNMClient nmClient;
-  private InetSocketAddress trackerAddr;
-  private URL trackerUrl;
   private Resource maxCapability;
 
   public Hadoop21YarnAMClient(Configuration conf) {
-    String masterContainerId = System.getenv().get(ApplicationConstants.Environment.CONTAINER_ID.name());
-    Preconditions.checkArgument(masterContainerId != null,
-                                "Missing %s from environment", ApplicationConstants.Environment.CONTAINER_ID.name());
-    this.containerId = ConverterUtils.toContainerId(masterContainerId);
-    this.containerRequests = ArrayListMultimap.create();
+    super(ApplicationConstants.Environment.CONTAINER_ID.name());
 
     this.amrmClient = AMRMClient.createAMRMClient();
     this.amrmClient.init(conf);
@@ -105,89 +90,50 @@ public final class Hadoop21YarnAMClient extends AbstractIdleService implements Y
   }
 
   @Override
-  public ContainerId getContainerId() {
-    return containerId;
+  public String getHost() {
+    return System.getenv().get(ApplicationConstants.Environment.NM_HOST.name());
   }
 
   @Override
-  public String getHost() {
-    return System.getenv().get(ApplicationConstants.Environment.NM_HOST.name());
+  protected AMRMClient.ContainerRequest createContainerRequest(Priority priority, Resource capability,
+                                                               @Nullable String[] hosts, @Nullable String[] racks) {
+    return new AMRMClient.ContainerRequest(capability, hosts, racks, priority);
   }
 
   @Override
-  public void setTracker(InetSocketAddress trackerAddr, URL trackerUrl) {
-    this.trackerAddr = trackerAddr;
-    this.trackerUrl = trackerUrl;
+  protected void addContainerRequest(AMRMClient.ContainerRequest request) {
+    amrmClient.addContainerRequest(request);
   }
 
   @Override
-  public synchronized void allocate(float progress, AllocateHandler handler) throws Exception {
+  protected void removeContainerRequest(AMRMClient.ContainerRequest request) {
+    amrmClient.removeContainerRequest(request);
+  }
+
+  @Override
+  protected AllocateResult doAllocate(float progress) throws Exception {
     AllocateResponse allocateResponse = amrmClient.allocate(progress);
-    List<ProcessLauncher<YarnContainerInfo>> launchers
+    List<RunnableProcessLauncher> launchers
       = Lists.newArrayListWithCapacity(allocateResponse.getAllocatedContainers().size());
 
     for (Container container : allocateResponse.getAllocatedContainers()) {
       launchers.add(new RunnableProcessLauncher(new Hadoop21YarnContainerInfo(container), nmClient));
     }
 
-    if (!launchers.isEmpty()) {
-      handler.acquired(launchers);
-
-      // If no process has been launched through the given launcher, return the container.
-      for (ProcessLauncher<YarnContainerInfo> l : launchers) {
-        // This cast always works.
-        RunnableProcessLauncher launcher = (RunnableProcessLauncher) l;
-        if (!launcher.isLaunched()) {
-          Container container = launcher.getContainerInfo().getContainer();
-          LOG.info("Nothing to run in container, releasing it: {}", container);
-          amrmClient.releaseAssignedContainer(container.getId());
-        }
-      }
-    }
-
     List<YarnContainerStatus> completed = ImmutableList.copyOf(
       Iterables.transform(allocateResponse.getCompletedContainersStatuses(), STATUS_TRANSFORM));
-    if (!completed.isEmpty()) {
-      handler.completed(completed);
-    }
-  }
 
-  @Override
-  public ContainerRequestBuilder addContainerRequest(Resource capability) {
-    return addContainerRequest(capability, 1);
+    return new AllocateResult(launchers, completed);
   }
 
   @Override
-  public ContainerRequestBuilder addContainerRequest(Resource capability, int count) {
-    return new ContainerRequestBuilder(adjustCapability(capability), count) {
-      @Override
-      public String apply() {
-        synchronized (Hadoop21YarnAMClient.this) {
-          String id = UUID.randomUUID().toString();
-
-          String[] hosts = this.hosts.isEmpty() ? null : this.hosts.toArray(new String[this.hosts.size()]);
-          String[] racks = this.racks.isEmpty() ? null : this.racks.toArray(new String[this.racks.size()]);
-
-          for (int i = 0; i < count; i++) {
-            AMRMClient.ContainerRequest request = new AMRMClient.ContainerRequest(capability, hosts, racks, priority);
-            containerRequests.put(id, request);
-            amrmClient.addContainerRequest(request);
-          }
-
-          return id;
-        }
-      }
-    };
+  protected void releaseAssignedContainer(YarnContainerInfo containerInfo) {
+    Container container = containerInfo.getContainer();
+    amrmClient.releaseAssignedContainer(container.getId());
   }
 
   @Override
-  public synchronized void completeContainerRequest(String id) {
-    for (AMRMClient.ContainerRequest request : containerRequests.removeAll(id)) {
-      amrmClient.removeContainerRequest(request);
-    }
-  }
-
-  private Resource adjustCapability(Resource resource) {
+  protected Resource adjustCapability(Resource resource) {
     int cores = resource.getVirtualCores();
     int updatedCores = Math.min(resource.getVirtualCores(), maxCapability.getVirtualCores());
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/2c3cf396/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
index f1ad20e..2b7f049 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
@@ -291,7 +291,7 @@ public final class ApplicationMasterService extends AbstractTwillService {
     final Set<String> ids = Sets.newHashSet(runningContainers.getContainerIds());
     YarnAMClient.AllocateHandler handler = new YarnAMClient.AllocateHandler() {
       @Override
-      public void acquired(List<ProcessLauncher<YarnContainerInfo>> launchers) {
+      public void acquired(List<? extends ProcessLauncher<YarnContainerInfo>> launchers) {
         // no-op
       }
 
@@ -352,7 +352,7 @@ public final class ApplicationMasterService extends AbstractTwillService {
 
     YarnAMClient.AllocateHandler allocateHandler = new YarnAMClient.AllocateHandler() {
       @Override
-      public void acquired(List<ProcessLauncher<YarnContainerInfo>> launchers) {
+      public void acquired(List<? extends ProcessLauncher<YarnContainerInfo>> launchers) {
         launchRunnable(launchers, provisioning);
       }
 
@@ -525,7 +525,7 @@ public final class ApplicationMasterService extends AbstractTwillService {
   /**
    * Launches runnables in the provisioned containers.
    */
-  private void launchRunnable(List<ProcessLauncher<YarnContainerInfo>> launchers,
+  private void launchRunnable(List<? extends ProcessLauncher<YarnContainerInfo>> launchers,
                               Queue<ProvisionRequest> provisioning) {
     for (ProcessLauncher<YarnContainerInfo> processLauncher : launchers) {
       LOG.info("Got container {}", processLauncher.getContainerInfo().getId());

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/2c3cf396/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnAMClient.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnAMClient.java
new file mode 100644
index 0000000..9718150
--- /dev/null
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/AbstractYarnAMClient.java
@@ -0,0 +1,229 @@
+/*
+ * 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.twill.internal.yarn;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import com.google.common.util.concurrent.AbstractIdleService;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.twill.internal.ProcessLauncher;
+import org.apache.twill.internal.appmaster.RunnableProcessLauncher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.util.List;
+import java.util.UUID;
+import javax.annotation.Nullable;
+
+/**
+ * Abstract base for implementing YarnAMClient for different versions of hadoop.
+ *
+ * @param <T> Type of container request.
+ */
+public abstract class AbstractYarnAMClient<T> extends AbstractIdleService implements YarnAMClient {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbstractYarnAMClient.class);
+
+  // Map from a unique ID to inflight requests
+  private final Multimap<String, T> containerRequests;
+
+  // List of requests pending to send through allocate call
+  private final List<T> requests;
+  // List of requests pending to remove through allocate call
+  private final List<T> removes;
+
+  protected final ContainerId containerId;
+  protected InetSocketAddress trackerAddr;
+  protected URL trackerUrl;
+
+  /**
+   * Constructs an instance of AMClient.
+   *
+   * @param containerIdEnvName Name of the environment variable that contains value of the AM container ID.
+   */
+  protected AbstractYarnAMClient(String containerIdEnvName) {
+    String masterContainerId = System.getenv().get(containerIdEnvName);
+    Preconditions.checkArgument(masterContainerId != null,
+                                "Missing %s from environment", containerIdEnvName);
+    this.containerId = ConverterUtils.toContainerId(masterContainerId);
+    this.containerRequests = ArrayListMultimap.create();
+    this.requests = Lists.newLinkedList();
+    this.removes = Lists.newLinkedList();
+  }
+
+
+  @Override
+  public final ContainerId getContainerId() {
+    return containerId;
+  }
+
+  @Override
+  public final void setTracker(InetSocketAddress trackerAddr, URL trackerUrl) {
+    this.trackerAddr = trackerAddr;
+    this.trackerUrl = trackerUrl;
+  }
+
+  @Override
+  public final synchronized void allocate(float progress, AllocateHandler handler) throws Exception {
+    // In one allocate cycle, either only do new container request or removal of requests.
+    // This is a workaround for YARN-314.
+    // When remove a container request, AMRMClient will send a container request with size = 0
+    // With bug YARN-314, if we mix the allocate call with new container request of the same priority,
+    // in some cases the RM would not see the new request (based on sorting of resource capability),
+    // but rather only see the one with size = 0.
+    if (removes.isEmpty()) {
+      for (T request : requests) {
+        addContainerRequest(request);
+      }
+      requests.clear();
+    } else {
+      for (T request : removes) {
+        removeContainerRequest(request);
+      }
+      removes.clear();
+    }
+
+    AllocateResult allocateResponse = doAllocate(progress);
+    List<RunnableProcessLauncher> launchers = allocateResponse.getLaunchers();
+
+    if (!launchers.isEmpty()) {
+      handler.acquired(launchers);
+
+      // If no process has been launched through the given launcher, return the container.
+      for (ProcessLauncher<YarnContainerInfo> l : launchers) {
+        // This cast always works.
+        RunnableProcessLauncher launcher = (RunnableProcessLauncher) l;
+        if (!launcher.isLaunched()) {
+          YarnContainerInfo containerInfo = launcher.getContainerInfo();
+          LOG.info("Nothing to run in container, releasing it: {}", containerInfo.getContainer());
+          releaseAssignedContainer(containerInfo);
+        }
+      }
+    }
+
+    List<YarnContainerStatus> completed = allocateResponse.getCompletedStatus();
+    if (!completed.isEmpty()) {
+      handler.completed(completed);
+    }
+  }
+
+  @Override
+  public final ContainerRequestBuilder addContainerRequest(Resource capability) {
+    return addContainerRequest(capability, 1);
+  }
+
+  @Override
+  public final ContainerRequestBuilder addContainerRequest(Resource capability, int count) {
+    return new ContainerRequestBuilder(adjustCapability(capability), count) {
+      @Override
+      public String apply() {
+        synchronized (AbstractYarnAMClient.this) {
+          String id = UUID.randomUUID().toString();
+
+          String[] hosts = this.hosts.isEmpty() ? null : this.hosts.toArray(new String[this.hosts.size()]);
+          String[] racks = this.racks.isEmpty() ? null : this.racks.toArray(new String[this.racks.size()]);
+
+          for (int i = 0; i < count; i++) {
+            T request = createContainerRequest(priority, capability, hosts, racks);
+            containerRequests.put(id, request);
+            requests.add(request);
+          }
+
+          return id;
+        }
+      }
+    };
+
+  }
+
+  @Override
+  public final synchronized void completeContainerRequest(String id) {
+    for (T request : containerRequests.removeAll(id)) {
+      removes.add(request);
+    }
+  }
+
+  /**
+   * Adjusts the given resource capability to fit in the cluster limit.
+   *
+   * @param capability The capability to be adjusted.
+   * @return A {@link Resource} instance representing the adjusted result.
+   */
+  protected abstract Resource adjustCapability(Resource capability);
+
+  /**
+   * Creates a container request based on the given requirement.
+   *
+   * @param priority The priority of the request.
+   * @param capability The resource capability.
+   * @param hosts Sets of hosts. Could be {@code null}.
+   * @param racks Sets of racks. Could be {@code null}.
+   * @return A container request.
+   */
+  protected abstract T createContainerRequest(Priority priority, Resource capability,
+                                              @Nullable String[] hosts, @Nullable String[] racks);
+
+  /**
+   * Adds the given request to prepare for next allocate call.
+   */
+  protected abstract void addContainerRequest(T request);
+
+  /**
+   * Removes the given request to prepare for the next allocate call.
+   */
+  protected abstract void removeContainerRequest(T request);
+
+  /**
+   * Performs actual allocate call to RM.
+   */
+  protected abstract AllocateResult doAllocate(float progress) throws Exception;
+
+  /**
+   * Releases the given container back to RM.
+   */
+  protected abstract void releaseAssignedContainer(YarnContainerInfo containerInfo);
+
+  /**
+   * Class for carrying results for the {@link #doAllocate(float)} call.
+   */
+  protected static final class AllocateResult {
+    private final List<RunnableProcessLauncher> launchers;
+    private final List<YarnContainerStatus> completedStatus;
+
+    public AllocateResult(List<RunnableProcessLauncher> launchers, List<YarnContainerStatus> completedStatus) {
+      this.launchers = ImmutableList.copyOf(launchers);
+      this.completedStatus = ImmutableList.copyOf(completedStatus);
+    }
+
+    public List<RunnableProcessLauncher> getLaunchers() {
+      return launchers;
+    }
+
+    public List<YarnContainerStatus> getCompletedStatus() {
+      return completedStatus;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/2c3cf396/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
index 370ca3c..6a5ee36 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
@@ -19,7 +19,6 @@ package org.apache.twill.internal.yarn;
 
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.Service;
-import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -29,7 +28,6 @@ import org.apache.twill.internal.ProcessLauncher;
 import java.net.InetSocketAddress;
 import java.net.URL;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
@@ -95,7 +93,7 @@ public interface YarnAMClient extends Service {
    */
   // TODO: Move AM heartbeat logic into this interface so AM only needs to handle callback.
   interface AllocateHandler {
-    void acquired(List<ProcessLauncher<YarnContainerInfo>> launchers);
+    void acquired(List<? extends ProcessLauncher<YarnContainerInfo>> launchers);
 
     void completed(List<YarnContainerStatus> completed);
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/2c3cf396/twill-yarn/src/test/java/org/apache/twill/yarn/ContainerSizeTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/ContainerSizeTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/ContainerSizeTestRun.java
new file mode 100644
index 0000000..42d332d
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/ContainerSizeTestRun.java
@@ -0,0 +1,119 @@
+/*
+ * 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.twill.yarn;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.twill.api.AbstractTwillRunnable;
+import org.apache.twill.api.ResourceSpecification;
+import org.apache.twill.api.TwillApplication;
+import org.apache.twill.api.TwillController;
+import org.apache.twill.api.TwillRunner;
+import org.apache.twill.api.TwillSpecification;
+import org.apache.twill.api.logging.PrinterLogHandler;
+import org.apache.twill.discovery.ServiceDiscovered;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.PrintWriter;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Test for requesting different container size in different order.
+ * It specifically test for workaround for YARN-314.
+ */
+public class ContainerSizeTestRun extends BaseYarnTest {
+
+  @Test
+  public void testContainerSize() throws InterruptedException {
+    TwillRunner runner = YarnTestUtils.getTwillRunner();
+    TwillController controller = runner.prepare(new SleepApp())
+      .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
+      .start();
+
+    try {
+      ServiceDiscovered discovered = controller.discoverService("sleep");
+      Assert.assertTrue(YarnTestUtils.waitForSize(discovered, 2, 60));
+    } finally {
+      controller.stopAndWait();
+    }
+  }
+
+
+  /**
+   * An application that has two runnables with different memory size.
+   */
+  public static final class SleepApp implements TwillApplication {
+
+    @Override
+    public TwillSpecification configure() {
+      ResourceSpecification largeRes = ResourceSpecification.Builder.with()
+        .setVirtualCores(1)
+        .setMemory(1024, ResourceSpecification.SizeUnit.MEGA)
+        .build();
+
+      ResourceSpecification smallRes = ResourceSpecification.Builder.with()
+        .setVirtualCores(1)
+        .setMemory(512, ResourceSpecification.SizeUnit.MEGA)
+        .build();
+
+      return TwillSpecification.Builder.with()
+        .setName("SleepApp")
+        .withRunnable()
+          .add("sleep1", new SleepRunnable(12345), largeRes).noLocalFiles()
+          .add("sleep2", new SleepRunnable(12346), smallRes).noLocalFiles()
+        .withOrder()
+          .begin("sleep1")
+          .nextWhenStarted("sleep2")
+        .build();
+    }
+  }
+
+
+  /**
+   * A runnable that sleep for 30 seconds.
+   */
+  public static final class SleepRunnable extends AbstractTwillRunnable {
+
+    private volatile Thread runThread;
+
+    public SleepRunnable(int port) {
+      super(ImmutableMap.of("port", Integer.toString(port)));
+    }
+
+    @Override
+    public void run() {
+      runThread = Thread.currentThread();
+      Random random = new Random();
+      getContext().announce("sleep", Integer.parseInt(getContext().getSpecification().getConfigs().get("port")));
+      try {
+        TimeUnit.SECONDS.sleep(30);
+      } catch (InterruptedException e) {
+        // Ignore.
+      }
+    }
+
+    @Override
+    public void stop() {
+      if (runThread != null) {
+        runThread.interrupt();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/2c3cf396/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
index 4d45ad1..f5ead6b 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
@@ -23,12 +23,14 @@ import org.apache.twill.api.TwillRunner;
 import org.apache.twill.api.logging.LogEntry;
 import org.apache.twill.api.logging.LogHandler;
 import org.apache.twill.api.logging.LogThrowable;
+import org.apache.twill.api.logging.PrinterLogHandler;
 import org.apache.twill.common.Services;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.PrintWriter;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.CountDownLatch;
@@ -62,6 +64,7 @@ public class LogHandlerTestRun extends BaseYarnTest {
 
     TwillRunner runner = YarnTestUtils.getTwillRunner();
     TwillController controller = runner.prepare(new LogRunnable())
+                                       .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
                                        .addLogHandler(logHandler)
                                        .start();
 


[38/50] [abbrv] git commit: (TWILL-57) Created BundledJarRunnable (in twill-ext) for running a bundled jar in its own class loader within a TwillApplication. Added twill-examples, which contains examples demonstrating Twill functionality.

Posted by ch...@apache.org.
(TWILL-57) Created BundledJarRunnable (in twill-ext) for running a bundled jar in its own class loader within a TwillApplication. Added twill-examples, which contains examples demonstrating Twill functionality.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: d4a352de4a62abdf4607af77d610597630e0ee1a
Parents: 875fbca
Author: Alvin Wang <al...@continuuity.com>
Authored: Thu Mar 20 12:48:18 2014 -0700
Committer: Terence Yim <te...@continuuity.com>
Committed: Thu Mar 20 16:22:56 2014 -0700

----------------------------------------------------------------------
 pom.xml                                         |   1 +
 .../java/org/apache/twill/api/LocalFile.java    |   3 +
 twill-examples/echo/pom.xml                     |  75 +++++
 .../echo/src/main/java/echo/EchoMain.java       |  53 ++++
 twill-examples/pom.xml                          |  40 +++
 twill-examples/yarn/pom.xml                     |  82 +++++
 .../twill/example/yarn/BundledJarExample.java   | 116 +++++++
 .../apache/twill/example/yarn/HelloWorld.java   |  86 +++++
 twill-ext/pom.xml                               |  57 ++++
 .../apache/twill/ext/BundledJarRunnable.java    | 131 ++++++++
 .../org/apache/twill/ext/BundledJarRunner.java  | 315 +++++++++++++++++++
 .../twill/ext/BundledJarRunnableTest.java       |  64 ++++
 12 files changed, 1023 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8e93850..b97bac1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -137,6 +137,7 @@
         <module>twill-discovery-core</module>
         <module>twill-core</module>
         <module>twill-yarn</module>
+        <module>twill-ext</module>
     </modules>
 
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/twill-api/src/main/java/org/apache/twill/api/LocalFile.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/LocalFile.java b/twill-api/src/main/java/org/apache/twill/api/LocalFile.java
index bcc3e13..43e33ad 100644
--- a/twill-api/src/main/java/org/apache/twill/api/LocalFile.java
+++ b/twill-api/src/main/java/org/apache/twill/api/LocalFile.java
@@ -39,6 +39,9 @@ public interface LocalFile {
    */
   long getSize();
 
+  /**
+   * Indicates whether this file is an archive. If true, the file is expanded after being copied to the container host.
+   */
   boolean isArchive();
 
   @Nullable

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/twill-examples/echo/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/echo/pom.xml b/twill-examples/echo/pom.xml
new file mode 100644
index 0000000..8e665ab
--- /dev/null
+++ b/twill-examples/echo/pom.xml
@@ -0,0 +1,75 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>twill-examples</artifactId>
+        <groupId>org.apache.twill.example</groupId>
+        <version>0.2.0-incubating-SNAPSHOT</version>
+    </parent>
+
+    <name>Twill examples: Echo</name>
+    <description>
+        Simple echo application that echos the command line arguments passed to it.
+        Used to demonstrate usage of BundledJarApplication to run applications that have
+        class conflicts with Twill.
+    </description>
+    <artifactId>twill-examples-echo</artifactId>
+    <packaging>bundle</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>16.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+            <version>1.7.5</version>
+        </dependency>
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+            <version>1.2.17</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+                <version>2.3.7</version>
+                <extensions>true</extensions>
+                <configuration>
+                    <instructions>
+                        <Embed-Dependency>*;inline=false</Embed-Dependency>
+                        <Embed-Transitive>true</Embed-Transitive>
+                        <Embed-Directory>lib</Embed-Directory>
+                    </instructions>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/twill-examples/echo/src/main/java/echo/EchoMain.java
----------------------------------------------------------------------
diff --git a/twill-examples/echo/src/main/java/echo/EchoMain.java b/twill-examples/echo/src/main/java/echo/EchoMain.java
new file mode 100644
index 0000000..a2cb4bd
--- /dev/null
+++ b/twill-examples/echo/src/main/java/echo/EchoMain.java
@@ -0,0 +1,53 @@
+/*
+ * 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 echo;
+
+import com.google.common.base.Converter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+
+/**
+ * Simple application used for BundledJarExample and RuntimeBundledJarExample.
+ */
+public class EchoMain {
+  private static final Logger logger = LoggerFactory.getLogger(EchoMain.class);
+
+  public static void main(String[] args) {
+    logger.info("Hello from EchoMain: " + new TestConverter().convert("sdflkj"));
+    System.err.println("err HELLO from scatch");
+    System.out.println("out HELLO from scatch");
+    logger.info("Got args: " + Arrays.toString(args));
+    System.out.println("Got args: " + Arrays.toString(args));
+  }
+
+  public static class TestConverter extends Converter<String, Integer> {
+
+    @Override
+    protected Integer doForward(String s) {
+      return s.length();
+    }
+
+    @Override
+    protected String doBackward(Integer integer) {
+      return integer.toString();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/twill-examples/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/pom.xml b/twill-examples/pom.xml
new file mode 100644
index 0000000..60f48fc
--- /dev/null
+++ b/twill-examples/pom.xml
@@ -0,0 +1,40 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+    <groupId>org.apache.twill.example</groupId>
+    <artifactId>twill-examples</artifactId>
+    <version>0.2.0-incubating-SNAPSHOT</version>
+    <name>Twill examples</name>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>echo</module>
+        <module>yarn</module>
+    </modules>
+
+    <properties>
+        <twill.version>0.2.0-incubating-SNAPSHOT</twill.version>
+        <twill.groupId>org.apache.twill</twill.groupId>
+    </properties>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/twill-examples/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/yarn/pom.xml b/twill-examples/yarn/pom.xml
new file mode 100644
index 0000000..3deea44
--- /dev/null
+++ b/twill-examples/yarn/pom.xml
@@ -0,0 +1,82 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>twill-examples</artifactId>
+        <groupId>org.apache.twill.example</groupId>
+        <version>0.2.0-incubating-SNAPSHOT</version>
+    </parent>
+
+    <name>Twill examples: YARN</name>
+    <description>Examples demonstrating usage of twill-yarn</description>
+    <artifactId>twill-examples-yarn</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>${twill.groupId}</groupId>
+            <artifactId>twill-yarn</artifactId>
+            <version>${twill.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>${twill.groupId}</groupId>
+            <artifactId>twill-ext</artifactId>
+            <version>${twill.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>13.0.1</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>2.1</version>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <createDependencyReducedPom>false</createDependencyReducedPom>
+                            <artifactSet>
+                                <excludes>
+                                    <exclude>org.apache.hadoop:*</exclude>
+                                </excludes>
+                            </artifactSet>
+                            <transformers>
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                </transformer>
+                            </transformers>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/twill-examples/yarn/src/main/java/org/apache/twill/example/yarn/BundledJarExample.java
----------------------------------------------------------------------
diff --git a/twill-examples/yarn/src/main/java/org/apache/twill/example/yarn/BundledJarExample.java b/twill-examples/yarn/src/main/java/org/apache/twill/example/yarn/BundledJarExample.java
new file mode 100644
index 0000000..3cc88a1
--- /dev/null
+++ b/twill-examples/yarn/src/main/java/org/apache/twill/example/yarn/BundledJarExample.java
@@ -0,0 +1,116 @@
+/*
+ * 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.twill.example.yarn;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.twill.api.*;
+import org.apache.twill.api.logging.PrinterLogHandler;
+import org.apache.twill.common.Services;
+import org.apache.twill.ext.BundledJarRunnable;
+import org.apache.twill.ext.BundledJarRunner;
+import org.apache.twill.yarn.YarnTwillRunnerService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.PrintWriter;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Demonstrates using BundledJarApplication to run a bundled jar
+ * as defined by {@link org.apache.twill.ext.BundledJarRunner}.
+ */
+public class BundledJarExample {
+  public static final Logger LOG = LoggerFactory.getLogger(BundledJarExample.class);
+
+  /**
+   * BundledJarApplication that specifies a single instance of main.sample.Scratch
+   * to be run from a bundled jar.
+   */
+  private static class ExampleBundledJarApp implements TwillApplication {
+    private final String jarName;
+    private final URI jarURI;
+
+    public ExampleBundledJarApp(String jarName, URI jarURI) {
+      this.jarName = jarName;
+      this.jarURI = jarURI;
+    }
+
+    @Override
+    public TwillSpecification configure() {
+      return TwillSpecification.Builder.with()
+        .setName("ExampleBundedJarApp")
+        .withRunnable()
+        .add("BundledJarRunnable", new BundledJarRunnable())
+        .withLocalFiles()
+        .add(jarName, jarURI, false)
+        .apply()
+        .anyOrder()
+        .build();
+    }
+  }
+
+  public static void main(String[] args) {
+    if (args.length < 3) {
+      System.err.println("Arguments format: <host:port of zookeeper server>"
+                           + " <bundle jar path> <main class name> <extra args>");
+      System.exit(1);
+    }
+
+    String zkStr = args[0];
+    BundledJarRunner.Arguments arguments = BundledJarRunner.Arguments.fromArray(
+      Arrays.copyOfRange(args, 1, args.length));
+
+    File jarFile = new File(arguments.getJarFileName());
+    Preconditions.checkArgument(jarFile != null);
+    Preconditions.checkState(jarFile.exists());
+    Preconditions.checkState(jarFile.canRead());
+
+    final TwillRunnerService twillRunner = new YarnTwillRunnerService(new YarnConfiguration(), zkStr);
+    twillRunner.startAndWait();
+
+
+    final TwillController controller = twillRunner.prepare(
+      new ExampleBundledJarApp(jarFile.getName(), jarFile.toURI()))
+      .withArguments("BundledJarRunnable", arguments.toArray())
+      .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
+      .start();
+
+    Runtime.getRuntime().addShutdownHook(new Thread() {
+      @Override
+      public void run() {
+        controller.stopAndWait();
+        twillRunner.stopAndWait();
+      }
+    });
+
+    try {
+      Services.getCompletionFuture(controller).get();
+    } catch (InterruptedException e) {
+      LOG.error("Error", e);
+    } catch (ExecutionException e) {
+      LOG.error("Error", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/twill-examples/yarn/src/main/java/org/apache/twill/example/yarn/HelloWorld.java
----------------------------------------------------------------------
diff --git a/twill-examples/yarn/src/main/java/org/apache/twill/example/yarn/HelloWorld.java b/twill-examples/yarn/src/main/java/org/apache/twill/example/yarn/HelloWorld.java
new file mode 100644
index 0000000..8dc83ff
--- /dev/null
+++ b/twill-examples/yarn/src/main/java/org/apache/twill/example/yarn/HelloWorld.java
@@ -0,0 +1,86 @@
+/*
+ * 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.twill.example.yarn;
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.twill.api.AbstractTwillRunnable;
+import org.apache.twill.api.TwillController;
+import org.apache.twill.api.TwillRunnerService;
+import org.apache.twill.api.logging.PrinterLogHandler;
+import org.apache.twill.common.Services;
+import org.apache.twill.yarn.YarnTwillRunnerService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintWriter;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Hello World example using twill-yarn to run a TwillApplication over YARN.
+ */
+public class HelloWorld {
+  public static Logger LOG = LoggerFactory.getLogger(HelloWorld.class);
+
+  /**
+   * Hello World runnable that is provided to TwillRunnerService to be run.
+   */
+  private static class HelloWorldRunnable extends AbstractTwillRunnable {
+    @Override
+    public void run() {
+      LOG.info("Hello World. My first distributed application.");
+    }
+
+    @Override
+    public void stop() {
+    }
+  }
+
+  public static void main(String[] args) {
+    if (args.length < 1) {
+      System.err.println("Arguments format: <host:port of zookeeper server>");
+    }
+
+    String zkStr = args[0];
+
+    final TwillRunnerService twillRunner =
+      new YarnTwillRunnerService(
+        new YarnConfiguration(), zkStr);
+    twillRunner.startAndWait();
+
+    final TwillController controller =
+      twillRunner.prepare(new HelloWorldRunnable())
+        .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
+        .start();
+
+    Runtime.getRuntime().addShutdownHook(new Thread() {
+      @Override
+      public void run() {
+        controller.stopAndWait();
+        twillRunner.stopAndWait();
+      }
+    });
+
+    try {
+      Services.getCompletionFuture(controller).get();
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    } catch (ExecutionException e) {
+      e.printStackTrace();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/twill-ext/pom.xml
----------------------------------------------------------------------
diff --git a/twill-ext/pom.xml b/twill-ext/pom.xml
new file mode 100644
index 0000000..d6245bf
--- /dev/null
+++ b/twill-ext/pom.xml
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>twill-parent</artifactId>
+        <groupId>org.apache.twill</groupId>
+        <version>0.2.0-incubating-SNAPSHOT</version>
+    </parent>
+
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>twill-ext</artifactId>
+    <name>Twill extensions</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>twill-yarn</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>twill-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.code.gson</groupId>
+            <artifactId>gson</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+        </dependency>
+    </dependencies>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/twill-ext/src/main/java/org/apache/twill/ext/BundledJarRunnable.java
----------------------------------------------------------------------
diff --git a/twill-ext/src/main/java/org/apache/twill/ext/BundledJarRunnable.java b/twill-ext/src/main/java/org/apache/twill/ext/BundledJarRunnable.java
new file mode 100644
index 0000000..a9d62e9
--- /dev/null
+++ b/twill-ext/src/main/java/org/apache/twill/ext/BundledJarRunnable.java
@@ -0,0 +1,131 @@
+/*
+ * 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.twill.ext;
+
+import com.google.common.base.Preconditions;
+import org.apache.twill.api.TwillContext;
+import org.apache.twill.api.TwillRunnable;
+import org.apache.twill.api.TwillRunnableSpecification;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+
+/**
+ * Uses {@link BundledJarRunnable} to run a bundled jar.
+ */
+public class BundledJarRunnable implements TwillRunnable {
+  private static final Logger LOG = LoggerFactory.getLogger(BundledJarRunnable.class);
+
+  /**
+   * Runs the bundled jar.
+   */
+  private BundledJarRunner jarRunner;
+
+  /**
+   * Arguments for running the bundled jar.
+   */
+  private BundledJarRunner.Arguments arguments;
+
+  @Override
+  public void stop() {
+    // stop
+  }
+
+  @Override
+  public void run() {
+    Preconditions.checkNotNull(jarRunner);
+    Preconditions.checkNotNull(arguments.getMainClassName());
+    Preconditions.checkNotNull(arguments.getMainArgs());
+
+    try {
+      jarRunner.run();
+    } catch (Throwable t) {
+      System.exit(1);
+    }
+  }
+
+  protected void doInitialize(TwillContext context) {
+    // NO-OP: left for subclass to implement
+  }
+
+  protected BundledJarRunner.Arguments getArguments() {
+    return arguments;
+  }
+
+  protected void setMainArgs(String[] mainArgs) {
+    this.arguments = new BundledJarRunner.Arguments.Builder()
+      .from(arguments)
+      .setMainArgs(mainArgs)
+      .createArguments();
+  }
+
+  protected void setMainArgs(String mainArgs) {
+    this.setMainArgs(mainArgs.split(" "));
+  }
+
+  private String getName() {
+    return getClass().getSimpleName();
+  }
+
+  @Override
+  public TwillRunnableSpecification configure() {
+    return TwillRunnableSpecification.Builder.with()
+      .setName(getName())
+      .noConfigs()
+      .build();
+  }
+
+  private BundledJarRunner loadJarRunner(File jarFile, BundledJarRunner.Arguments arguments) {
+    BundledJarRunner jarRunner = new BundledJarRunner(jarFile, arguments);
+
+    try {
+      jarRunner.load();
+      return jarRunner;
+    } catch (Exception e) {
+      LOG.error("Error loading classes into jarRunner", e);
+    }
+
+    return null;
+  }
+
+  @Override
+  public final void initialize(TwillContext context) {
+    this.doInitialize(context);
+
+    arguments = BundledJarRunner.Arguments.fromArray(context.getArguments());
+
+    File jarFile = new File(arguments.getJarFileName());
+    Preconditions.checkArgument(jarFile != null, "Jar file {} cannot be null", jarFile.getAbsolutePath());
+    Preconditions.checkArgument(jarFile.exists(), "Jar file {} must exist", jarFile.getAbsolutePath());
+    Preconditions.checkArgument(jarFile.canRead(), "Jar file {} must be readable", jarFile.getAbsolutePath());
+
+    jarRunner = loadJarRunner(jarFile, arguments);
+  }
+
+  @Override
+  public void handleCommand(org.apache.twill.api.Command command) throws Exception {
+    // No-op by default. Left for children class to override.
+  }
+
+  @Override
+  public void destroy() {
+    // No-op by default. Left for children class to override.
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/twill-ext/src/main/java/org/apache/twill/ext/BundledJarRunner.java
----------------------------------------------------------------------
diff --git a/twill-ext/src/main/java/org/apache/twill/ext/BundledJarRunner.java b/twill-ext/src/main/java/org/apache/twill/ext/BundledJarRunner.java
new file mode 100644
index 0000000..6b9a15d
--- /dev/null
+++ b/twill-ext/src/main/java/org/apache/twill/ext/BundledJarRunner.java
@@ -0,0 +1,315 @@
+/*
+ * 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.twill.ext;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.io.ByteStreams;
+import com.google.common.io.Files;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.Method;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Arrays;
+import java.util.Enumeration;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+
+/**
+ * Runs a bundled jar specified by jarPath.
+ *
+ * 1. Loads the bundled jar and its dependencies (in /lib) into a class loader
+ * assuming the following format of the bundled jar:
+ * /*.class (class files)
+ * /lib/*.jar (dependencies required by the user code)
+ *
+ * 2. Instantiates an instance of the class {#mainClassName} and calls main({#args}) on it.
+*/
+public class BundledJarRunner {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BundledJarRunner.class);
+
+  private final File jarFile;
+  private final Arguments arguments;
+  private Object mainObject;
+  private Method mainMethod;
+
+  public BundledJarRunner(File jarFile, Arguments arguments) {
+    Preconditions.checkNotNull(jarFile);
+    Preconditions.checkState(jarFile.exists());
+    Preconditions.checkState(jarFile.canRead());
+    Preconditions.checkNotNull(arguments.getMainClassName());
+    Preconditions.checkNotNull(arguments.getLibFolder());
+
+    this.jarFile = jarFile;
+    this.arguments = arguments;
+  }
+
+  public void load() throws ClassNotFoundException, NoSuchMethodException,
+    InstantiationException, IllegalAccessException, IOException {
+    this.load(ClassLoader.getSystemClassLoader());
+  }
+
+  public void load(ClassLoader parentClassLoader) throws IOException, ClassNotFoundException,
+    IllegalAccessException, InstantiationException, NoSuchMethodException {
+
+    String mainClassName = arguments.getMainClassName();
+    String libFolder = arguments.getLibFolder();
+
+    Preconditions.checkNotNull(mainClassName);
+    Preconditions.checkNotNull(libFolder);
+
+    File inputJarFile = this.jarFile;
+    File outputJarDir = Files.createTempDir();
+
+    LOG.debug("Unpacking jar to " + outputJarDir.getAbsolutePath());
+    JarFile jarFile = new JarFile(inputJarFile);
+    unJar(jarFile, outputJarDir);
+
+    LOG.debug("Loading jars into ClassLoader");
+    List<URL> classPathUrls = new LinkedList<URL>();
+    classPathUrls.add(inputJarFile.toURI().toURL());
+    classPathUrls.addAll(getJarURLs(new File(outputJarDir, libFolder)));
+    URL[] classPathUrlArray = classPathUrls.toArray(new URL[classPathUrls.size()]);
+
+    for (URL url : classPathUrlArray) {
+      LOG.debug("Loading jar: " + url.getPath());
+    }
+
+    ClassLoader classLoader = new URLClassLoader(classPathUrlArray, parentClassLoader);
+    Thread.currentThread().setContextClassLoader(classLoader);
+
+    LOG.debug("Instantiating instance of " + mainClassName);
+    Class<?> cls = classLoader.loadClass(mainClassName);
+    mainMethod = cls.getMethod("main", String[].class);
+    mainObject = cls.newInstance();
+  }
+
+  public void run() throws Throwable {
+    Preconditions.checkNotNull(mainMethod, "Must call load() first");
+    Preconditions.checkNotNull(mainObject, "Must call load() first");
+    String mainClassName = arguments.getMainClassName();
+    String[] args = arguments.getMainArgs();
+
+    try {
+      LOG.info("Invoking " + mainClassName + ".main(" + Arrays.toString(args) + ")");
+      mainMethod.invoke(mainObject, new Object[] { args });
+    } catch (Throwable t) {
+      LOG.error("Error while trying to run " + mainClassName + " within " + jarFile.getAbsolutePath(), t);
+      throw t;
+    }
+  }
+
+  private void unJar(JarFile jarFile, File targetDirectory) throws IOException {
+    Enumeration<JarEntry> entries = jarFile.entries();
+    while (entries.hasMoreElements()) {
+      JarEntry entry = entries.nextElement();
+      File output = new File(targetDirectory, entry.getName());
+
+      if (entry.isDirectory()) {
+        output.mkdirs();
+      } else {
+        output.getParentFile().mkdirs();
+
+        OutputStream os = new FileOutputStream(output);
+        try {
+          InputStream is = jarFile.getInputStream(entry);
+          try {
+            ByteStreams.copy(is, os);
+          } finally {
+            is.close();
+          }
+        } finally {
+          os.close();
+        }
+      }
+    }
+  }
+
+  private List<URL> getJarURLs(File dir) throws MalformedURLException {
+    File[] files = dir.listFiles(new FilenameFilter() {
+      @Override
+      public boolean accept(File dir, String name) {
+        return name.endsWith(".jar");
+      }
+    });
+    List<URL> urls = new LinkedList<URL>();
+
+    if (files != null) {
+      for (File file : files) {
+        urls.add(file.toURI().toURL());
+      }
+    } else {
+      LOG.warn("No jar files found in " + dir.getAbsolutePath());
+    }
+
+    return urls;
+  }
+
+  /**
+   * Contains runtime arguments for {@link org.apache.twill.ext.BundledJarRunner}.
+   */
+  public static class Arguments {
+
+    /**
+     * Filename of the bundled jar, as specified in the TwillSpecification local files.
+     */
+    private final String jarFileName;
+
+    /**
+     * Class name of the class having the main() that is to be called.
+     */
+    private final String mainClassName;
+
+    /**
+     * Arguments to pass the the main() of the class specified by mainClassName.
+     */
+    private final String[] mainArgs;
+
+    /**
+     * Folder within the bundled jar containing the jar dependencies.
+     */
+    private final String libFolder;
+
+    public Arguments(String jarFileName, String libFolder, String mainClassName, String[] mainArgs) {
+      this.jarFileName = jarFileName;
+      this.libFolder = libFolder;
+      this.mainClassName = mainClassName;
+      this.mainArgs = mainArgs;
+    }
+
+    public static Arguments fromArray(String[] args) {
+      Preconditions.checkArgument(args.length >= 3, "Requires at least 3 arguments:"
+        + " <jarFileName> <libFolder> <mainClassName>");
+
+      Builder builder = new Builder();
+      builder.setJarFileName(args[0]);
+      builder.setLibFolder(args[1]);
+      builder.setMainClassName(args[2]);
+      builder.setMainArgs(Arrays.copyOfRange(args, 3, args.length));
+
+      return builder.createArguments();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      Arguments arguments = (Arguments) o;
+      return Objects.equal(jarFileName, arguments.jarFileName)
+        && Objects.equal(libFolder, arguments.libFolder)
+        && Arrays.deepEquals(mainArgs, arguments.mainArgs)
+        && Objects.equal(mainClassName, arguments.mainClassName);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(jarFileName, mainClassName, mainArgs, libFolder);
+    }
+
+    public String[] toArray() {
+      String[] result = new String[3 + mainArgs.length];
+      result[0] = jarFileName;
+      result[1] = libFolder;
+      result[2] = mainClassName;
+      for (int i = 0; i < mainArgs.length; i++) {
+        result[3 + i] = mainArgs[i];
+      }
+
+      return result;
+    }
+
+    public String getJarFileName() {
+      return jarFileName;
+    }
+
+    public String getLibFolder() {
+      return libFolder;
+    }
+
+    public String getMainClassName() {
+      return mainClassName;
+    }
+
+    public String[] getMainArgs() {
+      return mainArgs;
+    }
+
+    /**
+     * Builder for {@link org.apache.twill.ext.BundledJarRunner.Arguments}.
+     */
+    public static class Builder {
+      private String jarFileName;
+      private String libFolder;
+      private String mainClassName;
+      private String[] mainArgs;
+
+      public Builder() {}
+
+      public Builder setJarFileName(String jarFileName) {
+        this.jarFileName = jarFileName;
+        return this;
+      }
+
+      public Builder setLibFolder(String libFolder) {
+        this.libFolder = libFolder;
+        return this;
+      }
+
+      public Builder setMainClassName(String mainClassName) {
+        this.mainClassName = mainClassName;
+        return this;
+      }
+
+      public Builder setMainArgs(String[] mainArgs) {
+        this.mainArgs = mainArgs;
+        return this;
+      }
+
+      public Builder from(Arguments arguments) {
+        this.jarFileName = arguments.getJarFileName();
+        this.libFolder = arguments.getLibFolder();
+        this.mainClassName = arguments.getMainClassName();
+        this.mainArgs = arguments.getMainArgs();
+        return this;
+      }
+
+      public Arguments createArguments() {
+        return new Arguments(jarFileName, libFolder, mainClassName, mainArgs);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d4a352de/twill-ext/src/test/java/org/apache/twill/ext/BundledJarRunnableTest.java
----------------------------------------------------------------------
diff --git a/twill-ext/src/test/java/org/apache/twill/ext/BundledJarRunnableTest.java b/twill-ext/src/test/java/org/apache/twill/ext/BundledJarRunnableTest.java
new file mode 100644
index 0000000..97b4fdd
--- /dev/null
+++ b/twill-ext/src/test/java/org/apache/twill/ext/BundledJarRunnableTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.twill.ext;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Tests for {@link BundledJarRunnable}.
+ */
+public class BundledJarRunnableTest {
+
+  @Test
+  public void testArgumentsOperations() {
+    String[] mainArgs = {"a", "b", "c"};
+    String jarFileName = "test/123.jar";
+    String libFolder = "libsdf";
+    String mainClassName = "org.apache.twill.ext.Test";
+
+    List<String> argsArrayList = ImmutableList.<String>builder()
+      .add(jarFileName)
+      .add(libFolder)
+      .add(mainClassName)
+      .addAll(Lists.newArrayList(mainArgs))
+      .build();
+
+    String[] argsArray = argsArrayList.toArray(new String[argsArrayList.size()]);
+
+    BundledJarRunner.Arguments args = new BundledJarRunner.Arguments.Builder()
+      .setMainArgs(mainArgs)
+      .setJarFileName(jarFileName)
+      .setLibFolder(libFolder)
+      .setMainClassName(mainClassName)
+      .createArguments();
+
+    Assert.assertArrayEquals(mainArgs, args.getMainArgs());
+    Assert.assertEquals(jarFileName, args.getJarFileName());
+    Assert.assertEquals(libFolder, args.getLibFolder());
+    Assert.assertEquals(mainClassName, args.getMainClassName());
+
+    String[] array = args.toArray();
+    Assert.assertArrayEquals(argsArray, array);
+    Assert.assertEquals(args, BundledJarRunner.Arguments.fromArray(array));
+  }
+}


[37/50] [abbrv] git commit: (TWILL-53) Expose DiscoveryServiceClient through TwillContext to allow TwillRunnable able to discover services announced within the same TwillApplication.

Posted by ch...@apache.org.
(TWILL-53) Expose DiscoveryServiceClient through TwillContext to allow TwillRunnable able to discover services announced within the same TwillApplication.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 875fbca941a8e9a24d29d934a9c550c25ed722d9
Parents: 3d3c0e9
Author: Terence Yim <te...@continuuity.com>
Authored: Tue Mar 11 16:45:17 2014 -0700
Committer: Terence Yim <te...@continuuity.com>
Committed: Wed Mar 12 13:14:16 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/twill/api/TwillContext.java |  14 +-
 .../twill/internal/BasicTwillContext.java       |  12 +-
 .../twill/discovery/DiscoveryServiceClient.java |   4 +-
 .../internal/container/TwillContainerMain.java  |   5 +-
 .../apache/twill/yarn/ServiceDiscoveryTest.java | 129 +++++++++++++++++++
 .../org/apache/twill/yarn/YarnTestSuite.java    |   3 +-
 6 files changed, 159 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/875fbca9/twill-api/src/main/java/org/apache/twill/api/TwillContext.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/TwillContext.java b/twill-api/src/main/java/org/apache/twill/api/TwillContext.java
index b4ddb6e..f7a7ac1 100644
--- a/twill-api/src/main/java/org/apache/twill/api/TwillContext.java
+++ b/twill-api/src/main/java/org/apache/twill/api/TwillContext.java
@@ -17,12 +17,15 @@
  */
 package org.apache.twill.api;
 
+import org.apache.twill.discovery.DiscoveryServiceClient;
+import org.apache.twill.discovery.ServiceDiscovered;
+
 import java.net.InetAddress;
 
 /**
  * Represents the runtime context of a {@link TwillRunnable}.
  */
-public interface TwillContext extends ServiceAnnouncer {
+public interface TwillContext extends ServiceAnnouncer, DiscoveryServiceClient {
 
   /**
    * Returns the {@link RunId} of this running instance of {@link TwillRunnable}.
@@ -73,4 +76,13 @@ public interface TwillContext extends ServiceAnnouncer {
    * Returns the amount of memory in MB the runnable is allowed to use.
    */
   int getMaxMemoryMB();
+
+  /**
+   * Discover service with the given name that is announced within the same {@link TwillApplication}.
+   *
+   * @param name Name of the service
+   * @return A {@link ServiceDiscovered} object representing the result.
+   */
+  @Override
+  ServiceDiscovered discover(String name);
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/875fbca9/twill-core/src/main/java/org/apache/twill/internal/BasicTwillContext.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/BasicTwillContext.java b/twill-core/src/main/java/org/apache/twill/internal/BasicTwillContext.java
index 61bdaef..4a503e0 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/BasicTwillContext.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/BasicTwillContext.java
@@ -23,6 +23,8 @@ import org.apache.twill.api.TwillRunnableSpecification;
 import org.apache.twill.common.Cancellable;
 import org.apache.twill.discovery.Discoverable;
 import org.apache.twill.discovery.DiscoveryService;
+import org.apache.twill.discovery.DiscoveryServiceClient;
+import org.apache.twill.discovery.ServiceDiscovered;
 
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -40,12 +42,14 @@ public final class BasicTwillContext implements TwillContext {
   private final TwillRunnableSpecification spec;
   private final int instanceId;
   private final DiscoveryService discoveryService;
+  private final DiscoveryServiceClient discoveryServiceClient;
   private final int allowedMemoryMB;
   private final int virtualCores;
   private volatile int instanceCount;
 
   public BasicTwillContext(RunId runId, RunId appRunId, InetAddress host, String[] args, String[] appArgs,
-                           TwillRunnableSpecification spec, int instanceId, DiscoveryService discoveryService,
+                           TwillRunnableSpecification spec, int instanceId,
+                           DiscoveryService discoveryService, DiscoveryServiceClient discoveryServiceClient,
                            int instanceCount, int allowedMemoryMB, int virtualCores) {
     this.runId = runId;
     this.appRunId = appRunId;
@@ -55,6 +59,7 @@ public final class BasicTwillContext implements TwillContext {
     this.spec = spec;
     this.instanceId = instanceId;
     this.discoveryService = discoveryService;
+    this.discoveryServiceClient = discoveryServiceClient;
     this.instanceCount = instanceCount;
     this.allowedMemoryMB = allowedMemoryMB;
     this.virtualCores = virtualCores;
@@ -128,4 +133,9 @@ public final class BasicTwillContext implements TwillContext {
       }
     });
   }
+
+  @Override
+  public ServiceDiscovered discover(String name) {
+    return discoveryServiceClient.discover(name);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/875fbca9/twill-discovery-api/src/main/java/org/apache/twill/discovery/DiscoveryServiceClient.java
----------------------------------------------------------------------
diff --git a/twill-discovery-api/src/main/java/org/apache/twill/discovery/DiscoveryServiceClient.java b/twill-discovery-api/src/main/java/org/apache/twill/discovery/DiscoveryServiceClient.java
index a58c83d..1298a16 100644
--- a/twill-discovery-api/src/main/java/org/apache/twill/discovery/DiscoveryServiceClient.java
+++ b/twill-discovery-api/src/main/java/org/apache/twill/discovery/DiscoveryServiceClient.java
@@ -23,10 +23,10 @@ package org.apache.twill.discovery;
 public interface DiscoveryServiceClient {
 
   /**
-   * Retrieves a list of {@link Discoverable} for the a service with the given name.
+   * Discover service with the given name.
    *
    * @param name Name of the service
-   * @return A {@link org.apache.twill.discovery.ServiceDiscovered} object representing the result.
+   * @return A {@link ServiceDiscovered} object representing the result.
    */
   ServiceDiscovered discover(String name);
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/875fbca9/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
index 072bbb0..c3aece6 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerMain.java
@@ -31,7 +31,6 @@ import org.apache.twill.api.RunId;
 import org.apache.twill.api.RuntimeSpecification;
 import org.apache.twill.api.TwillRunnableSpecification;
 import org.apache.twill.api.TwillSpecification;
-import org.apache.twill.discovery.DiscoveryService;
 import org.apache.twill.discovery.ZKDiscoveryService;
 import org.apache.twill.internal.Arguments;
 import org.apache.twill.internal.BasicTwillContext;
@@ -86,7 +85,7 @@ public final class TwillContainerMain extends ServiceMain {
         ZKClients.retryOnFailure(ZKClientService.Builder.of(zkConnectStr).build(),
                                  RetryStrategies.fixDelay(1, TimeUnit.SECONDS))));
 
-    DiscoveryService discoveryService = new ZKDiscoveryService(zkClientService);
+    ZKDiscoveryService discoveryService = new ZKDiscoveryService(zkClientService);
 
     TwillSpecification twillSpec = loadTwillSpec(twillSpecFile);
     renameLocalFiles(twillSpec.getRunnables().get(runnableName));
@@ -98,7 +97,7 @@ public final class TwillContainerMain extends ServiceMain {
       runId, appRunId, containerInfo.getHost(),
       arguments.getRunnableArguments().get(runnableName).toArray(new String[0]),
       arguments.getArguments().toArray(new String[0]),
-      runnableSpec, instanceId, discoveryService, instanceCount,
+      runnableSpec, instanceId, discoveryService, discoveryService, instanceCount,
       containerInfo.getMemoryMB(), containerInfo.getVirtualCores()
     );
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/875fbca9/twill-yarn/src/test/java/org/apache/twill/yarn/ServiceDiscoveryTest.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/ServiceDiscoveryTest.java b/twill-yarn/src/test/java/org/apache/twill/yarn/ServiceDiscoveryTest.java
new file mode 100644
index 0000000..77bc181
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/ServiceDiscoveryTest.java
@@ -0,0 +1,129 @@
+/*
+ * 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.twill.yarn;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.Service;
+import org.apache.twill.api.AbstractTwillRunnable;
+import org.apache.twill.api.TwillApplication;
+import org.apache.twill.api.TwillContext;
+import org.apache.twill.api.TwillController;
+import org.apache.twill.api.TwillRunner;
+import org.apache.twill.api.TwillSpecification;
+import org.apache.twill.api.logging.PrinterLogHandler;
+import org.apache.twill.common.Services;
+import org.apache.twill.common.Threads;
+import org.apache.twill.discovery.Discoverable;
+import org.apache.twill.discovery.ServiceDiscovered;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintWriter;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Test for ability to discover existence of services through {@link TwillContext}.
+ */
+public final class ServiceDiscoveryTest extends BaseYarnTest {
+
+  @Test
+  public void testServiceDiscovery() throws InterruptedException, ExecutionException, TimeoutException {
+    TwillRunner twillRunner = YarnTestUtils.getTwillRunner();
+    TwillController controller = twillRunner
+      .prepare(new ServiceApplication())
+      .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
+      .withArguments("r1", "12345")
+      .withArguments("r2", "45678")
+      .start();
+
+    ListenableFuture<Service.State> completion = Services.getCompletionFuture(controller);
+    try {
+      completion.get(60, TimeUnit.SECONDS);
+    } finally {
+      controller.stopAndWait();
+    }
+  }
+
+  /**
+   * An application that contains two {@link ServiceRunnable}.
+   */
+  public static final class ServiceApplication implements TwillApplication {
+
+    @Override
+    public TwillSpecification configure() {
+      return TwillSpecification.Builder.with()
+        .setName("ServiceApp")
+        .withRunnable()
+          .add("r1", new ServiceRunnable()).noLocalFiles()
+          .add("r2", new ServiceRunnable()).noLocalFiles()
+        .anyOrder()
+        .build();
+    }
+  }
+
+  /**
+   * A Runnable that will announce on service and wait for announcement from another instance in the same service.
+   */
+  public static final class ServiceRunnable extends AbstractTwillRunnable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ServiceRunnable.class);
+    private static final String SERVICE_NAME = "service";
+    private volatile Thread runThread;
+
+    @Override
+    public void run() {
+      this.runThread = Thread.currentThread();
+      final int port = Integer.parseInt(getContext().getArguments()[0]);
+      getContext().announce(SERVICE_NAME, port);
+
+      final CountDownLatch discoveredLatch = new CountDownLatch(1);
+
+      ServiceDiscovered serviceDiscovered = getContext().discover(SERVICE_NAME);
+      serviceDiscovered.watchChanges(new ServiceDiscovered.ChangeListener() {
+        @Override
+        public void onChange(ServiceDiscovered serviceDiscovered) {
+          // Try to find a discoverable that is not this instance
+          for (Discoverable discoverable : serviceDiscovered) {
+            int discoveredPort = discoverable.getSocketAddress().getPort();
+            if (SERVICE_NAME.equals(discoverable.getName()) && discoveredPort != port) {
+              LOG.info("{}: Service discovered at {}", getContext().getSpecification().getName(), discoveredPort);
+              discoveredLatch.countDown();
+            }
+          }
+        }
+      }, Threads.SAME_THREAD_EXECUTOR);
+
+      try {
+        discoveredLatch.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted.", e);
+      }
+    }
+
+    @Override
+    public void stop() {
+      if (runThread != null) {
+        runThread.interrupt();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/875fbca9/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
index 56fb6a5..51b6abf 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
@@ -33,7 +33,8 @@ import org.junit.runners.Suite;
                       FailureRestartTestRun.class,
                       ProvisionTimeoutTestRun.class,
                       LogHandlerTestRun.class,
-                      SessionExpireTestRun.class
+                      SessionExpireTestRun.class,
+                      ServiceDiscoveryTest.class
                     })
 public final class YarnTestSuite {
 


[28/50] [abbrv] git commit: (TWILL-47) Fix the tracking url registration

Posted by ch...@apache.org.
(TWILL-47) Fix the tracking url registration

1. Register the tracking URL without http:// for yarn-2.0.x.
2. Register without the "/resources" in path.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 9171d22c5293ff6edc271d6a81194d2c6efdf9e0
Parents: e5c62e5
Author: Terence Yim <te...@continuuity.com>
Authored: Thu Feb 13 14:45:12 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Feb 17 09:54:22 2014 -0800

----------------------------------------------------------------------
 .../org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java    | 5 ++++-
 .../org/apache/twill/internal/appmaster/TrackerService.java     | 3 +--
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9171d22c/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
index e773819..68d073d 100644
--- a/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
+++ b/twill-yarn/src/main/hadoop20/org/apache/twill/internal/yarn/Hadoop20YarnAMClient.java
@@ -93,9 +93,12 @@ public final class Hadoop20YarnAMClient extends AbstractIdleService implements Y
 
     amrmClient.start();
 
+    String url = String.format("%s:%d",
+                               trackerUrl.getHost(),
+                               trackerUrl.getPort() == -1 ? trackerUrl.getDefaultPort() : trackerUrl.getPort());
     RegisterApplicationMasterResponse response = amrmClient.registerApplicationMaster(trackerAddr.getHostName(),
                                                                                       trackerAddr.getPort(),
-                                                                                      trackerUrl.toString());
+                                                                                      url);
     maxCapability = response.getMaximumResourceCapability();
     minCapability = response.getMinimumResourceCapability();
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9171d22c/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
index 8ff5184..a9553c9 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
@@ -145,8 +145,7 @@ public final class TrackerService extends AbstractIdleService {
 
     Channel channel = bootstrap.bind(new InetSocketAddress(host, 0));
     bindAddress = (InetSocketAddress) channel.getLocalAddress();
-    url = URI.create(String.format("http://%s:%d", host, bindAddress.getPort()))
-             .resolve(TrackerService.PATH).toURL();
+    url = URI.create(String.format("http://%s:%d", host, bindAddress.getPort())).toURL();
     channelGroup.add(channel);
   }
 


[43/50] [abbrv] git commit: Bump version to 0.3.0-SNAPSHOT

Posted by ch...@apache.org.
Bump version to 0.3.0-SNAPSHOT


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

Branch: refs/heads/site
Commit: 6553ed07c40de838b56c3a3e4d0da0a9ac22cba8
Parents: 62e51eb
Author: Terence Yim <ch...@apache.org>
Authored: Fri Apr 11 23:53:27 2014 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Sat Apr 12 13:03:04 2014 -0700

----------------------------------------------------------------------
 pom.xml                      | 2 +-
 twill-api/pom.xml            | 2 +-
 twill-common/pom.xml         | 2 +-
 twill-core/pom.xml           | 2 +-
 twill-discovery-api/pom.xml  | 2 +-
 twill-discovery-core/pom.xml | 2 +-
 twill-ext/pom.xml            | 2 +-
 twill-yarn/pom.xml           | 2 +-
 twill-zookeeper/pom.xml      | 2 +-
 9 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6553ed07/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b97bac1..1481d54 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
 
     <groupId>org.apache.twill</groupId>
     <artifactId>twill-parent</artifactId>
-    <version>0.2.0-incubating-SNAPSHOT</version>
+    <version>0.3.0-incubating-SNAPSHOT</version>
     <packaging>pom</packaging>
     <name>Apache Twill</name>
     <url>http://twill.incubator.apache.org</url>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6553ed07/twill-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-api/pom.xml b/twill-api/pom.xml
index 338a716..250289c 100644
--- a/twill-api/pom.xml
+++ b/twill-api/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.twill</groupId>
         <artifactId>twill-parent</artifactId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
 
     <artifactId>twill-api</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6553ed07/twill-common/pom.xml
----------------------------------------------------------------------
diff --git a/twill-common/pom.xml b/twill-common/pom.xml
index 7506d84..e9e641f 100644
--- a/twill-common/pom.xml
+++ b/twill-common/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6553ed07/twill-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-core/pom.xml b/twill-core/pom.xml
index 124547c..7b7a085 100644
--- a/twill-core/pom.xml
+++ b/twill-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6553ed07/twill-discovery-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-api/pom.xml b/twill-discovery-api/pom.xml
index 2a12f39..0ce6c38 100644
--- a/twill-discovery-api/pom.xml
+++ b/twill-discovery-api/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6553ed07/twill-discovery-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-core/pom.xml b/twill-discovery-core/pom.xml
index 5570986..2bdad6b 100644
--- a/twill-discovery-core/pom.xml
+++ b/twill-discovery-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6553ed07/twill-ext/pom.xml
----------------------------------------------------------------------
diff --git a/twill-ext/pom.xml b/twill-ext/pom.xml
index d6245bf..3d835bf 100644
--- a/twill-ext/pom.xml
+++ b/twill-ext/pom.xml
@@ -22,7 +22,7 @@ limitations under the License.
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
 
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6553ed07/twill-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-yarn/pom.xml b/twill-yarn/pom.xml
index de45335..014489c 100644
--- a/twill-yarn/pom.xml
+++ b/twill-yarn/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6553ed07/twill-zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/twill-zookeeper/pom.xml b/twill-zookeeper/pom.xml
index 70de6bf..8b1f4fe 100644
--- a/twill-zookeeper/pom.xml
+++ b/twill-zookeeper/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 


[04/50] [abbrv] [TWILL-27] Upgrade to Kafka-0.8

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaConsumer.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaConsumer.java b/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaConsumer.java
new file mode 100644
index 0000000..d53ee98
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaConsumer.java
@@ -0,0 +1,92 @@
+/*
+ * 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.twill.kafka.client;
+
+import org.apache.twill.common.Cancellable;
+
+import java.util.Iterator;
+
+/**
+ * A consumer for consuming (reading) messages published to a Kafka server cluster.
+ */
+public interface KafkaConsumer {
+
+  /**
+   * Callback for receiving new messages.
+   */
+  interface MessageCallback {
+
+    /**
+     * Invoked when new messages is available.
+     * @param messages Iterator of new messages. The {@link FetchedMessage} instance maybe reused in the Iterator
+     *                 and across different invocation.
+     */
+    void onReceived(Iterator<FetchedMessage> messages);
+
+    /**
+     * Invoked when message consumption is stopped. When this method is invoked,
+     * no more {@link #onReceived(java.util.Iterator)} will get triggered.
+     */
+    void finished();
+  }
+
+  /**
+   * A builder for preparing message consumption.
+   */
+  interface Preparer {
+
+    /**
+     * Consumes messages from a given offset. If the given offset is invalid, it'll start consuming from the
+     * latest offset.
+     * @param topic Topic to consume from.
+     * @param partition Partition in the topic to consume from.
+     * @param offset Offset to starts with.
+     * @return This {@link Preparer} instance.
+     */
+    Preparer add(String topic, int partition, long offset);
+
+    /**
+     * Consumes messages from the earliest message available.
+     * @param topic Topic to consume from.
+     * @param partition Partition in the topic to consume from.
+     * @return This {@link Preparer} instance.
+     */
+    Preparer addFromBeginning(String topic, int partition);
+
+    /**
+     * Consumes messages from the latest message.
+     * @param topic Topic to consume from.
+     * @param partition Partition in the topic to consume from.
+     * @return This {@link Preparer} instance.
+     */
+    Preparer addLatest(String topic, int partition);
+
+    /**
+     * Starts the consumption as being configured by this {@link Preparer}.
+     * @param callback The {@link MessageCallback} for receiving new messages.
+     * @return A {@link Cancellable} for cancelling message consumption.
+     */
+    Cancellable consume(MessageCallback callback);
+  }
+
+  /**
+   * Prepares for message consumption.
+   * @return A {@link Preparer} to setup details about message consumption.
+   */
+  Preparer prepare();
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaPublisher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaPublisher.java b/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaPublisher.java
new file mode 100644
index 0000000..bffce97
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaPublisher.java
@@ -0,0 +1,95 @@
+/*
+ * 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.twill.kafka.client;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This interface is for publishing data to Kafka.
+ */
+public interface KafkaPublisher {
+
+  /**
+   * A Preparer for preparing to publish messages to a given topic. An instance of this class could be reused
+   * to send more messages after {@link #send()} is called.
+   */
+  interface Preparer {
+    /**
+     * Adds the given message to the message set, partitioned with the given partition key.
+     * @param message Remaining bytes in the ByteBuffer will be used as message payload. This method would
+     *                consume the ByteBuffer, meaning after this method returns, the remaining bytes in the
+     *                ByteBuffer would be {@code 0}.
+     * @param partitionKey Key for computing the partition Id to publish to. The {@link Object#hashCode()} method
+     *                     will be invoke to compute the id.
+     * @return
+     */
+    Preparer add(ByteBuffer message, Object partitionKey);
+
+    /**
+     * Sends all the messages being added through the {@link #add} method.
+     *
+     * @return A {@link ListenableFuture} that will be completed when the send action is done. If publish is succeeded,
+     *         it returns number of messages published, otherwise the failure reason will be carried in the future.
+     *         The {@link ListenableFuture#cancel(boolean)} method has no effect on the publish action.
+     */
+    ListenableFuture<Integer> send();
+  }
+
+  /**
+   * Represents the desired level of publish acknowledgment.
+   */
+  enum Ack {
+    /**
+     * Not wait for ack.
+     */
+    FIRE_AND_FORGET(0),
+
+    /**
+     * Waits for the leader received data.
+     */
+    LEADER_RECEIVED(1),
+
+    /**
+     * Waits for all replicas received data.
+     */
+    ALL_RECEIVED(-1);
+
+    private final int ack;
+
+    private Ack(int ack) {
+      this.ack = ack;
+    }
+
+    /**
+     * Returns the numerical ack number as understand by Kafka server.
+     */
+    public int getAck() {
+      return ack;
+    }
+  }
+
+  /**
+   * Prepares to publish to a given topic.
+   *
+   * @param topic Name of the topic.
+   * @return A {@link Preparer} to prepare for publishing.
+   */
+  Preparer prepare(String topic);
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/PreparePublish.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/PreparePublish.java b/twill-core/src/main/java/org/apache/twill/kafka/client/PreparePublish.java
deleted file mode 100644
index 5db4abb..0000000
--- a/twill-core/src/main/java/org/apache/twill/kafka/client/PreparePublish.java
+++ /dev/null
@@ -1,34 +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.twill.kafka.client;
-
-import com.google.common.util.concurrent.ListenableFuture;
-
-import java.nio.ByteBuffer;
-
-/**
- * This interface is for preparing to publish a set of messages to kafka.
- */
-public interface PreparePublish {
-
-  PreparePublish add(byte[] payload, Object partitionKey);
-
-  PreparePublish add(ByteBuffer payload, Object partitionKey);
-
-  ListenableFuture<?> publish();
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/TopicPartition.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/TopicPartition.java b/twill-core/src/main/java/org/apache/twill/kafka/client/TopicPartition.java
new file mode 100644
index 0000000..87040be
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/kafka/client/TopicPartition.java
@@ -0,0 +1,70 @@
+/*
+ * 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.twill.kafka.client;
+
+import com.google.common.base.Objects;
+
+/**
+ * Represents a combination of topic and partition.
+ */
+public class TopicPartition {
+
+  private final String topic;
+  private final int partition;
+
+  public TopicPartition(String topic, int partition) {
+    this.topic = topic;
+    this.partition = partition;
+  }
+
+  public final String getTopic() {
+    return topic;
+  }
+
+  public final int getPartition() {
+    return partition;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    TopicPartition other = (TopicPartition) o;
+    return partition == other.partition && topic.equals(other.topic);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = topic.hashCode();
+    result = 31 * result + partition;
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return Objects.toStringHelper(this)
+      .add("topic", topic)
+      .add("partition", partition)
+      .toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/package-info.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/package-info.java b/twill-core/src/main/java/org/apache/twill/kafka/client/package-info.java
index ea3bf20..cc22d12 100644
--- a/twill-core/src/main/java/org/apache/twill/kafka/client/package-info.java
+++ b/twill-core/src/main/java/org/apache/twill/kafka/client/package-info.java
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * This package provides a pure java Kafka client interface.
+ * This package provides Kafka client interfaces.
  */
 package org.apache.twill.kafka.client;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/resources/kafka-0.7.2.tgz
----------------------------------------------------------------------
diff --git a/twill-core/src/main/resources/kafka-0.7.2.tgz b/twill-core/src/main/resources/kafka-0.7.2.tgz
deleted file mode 100644
index 24178d9..0000000
Binary files a/twill-core/src/main/resources/kafka-0.7.2.tgz and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/test/java/org/apache/twill/kafka/client/KafkaTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/kafka/client/KafkaTest.java b/twill-core/src/test/java/org/apache/twill/kafka/client/KafkaTest.java
index 40fc3ed..9308fb0 100644
--- a/twill-core/src/test/java/org/apache/twill/kafka/client/KafkaTest.java
+++ b/twill-core/src/test/java/org/apache/twill/kafka/client/KafkaTest.java
@@ -17,24 +17,16 @@
  */
 package org.apache.twill.kafka.client;
 
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.Futures;
+import org.apache.twill.common.Cancellable;
 import org.apache.twill.common.Services;
 import org.apache.twill.internal.kafka.EmbeddedKafkaServer;
-import org.apache.twill.internal.kafka.client.Compression;
-import org.apache.twill.internal.kafka.client.SimpleKafkaClient;
+import org.apache.twill.internal.kafka.client.ZKKafkaClientService;
 import org.apache.twill.internal.utils.Networks;
 import org.apache.twill.internal.zookeeper.InMemoryZKServer;
 import org.apache.twill.zookeeper.ZKClientService;
-import com.google.common.base.Charsets;
-import com.google.common.base.Preconditions;
-import com.google.common.io.ByteStreams;
-import com.google.common.io.Files;
-import com.google.common.util.concurrent.Futures;
-import org.apache.commons.compress.archivers.ArchiveEntry;
-import org.apache.commons.compress.archivers.ArchiveException;
-import org.apache.commons.compress.archivers.ArchiveInputStream;
-import org.apache.commons.compress.archivers.ArchiveStreamFactory;
-import org.apache.commons.compress.compressors.CompressorException;
-import org.apache.commons.compress.compressors.CompressorStreamFactory;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -44,11 +36,10 @@ import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.Iterator;
 import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
 /**
@@ -64,7 +55,7 @@ public class KafkaTest {
   private static InMemoryZKServer zkServer;
   private static EmbeddedKafkaServer kafkaServer;
   private static ZKClientService zkClientService;
-  private static KafkaClient kafkaClient;
+  private static KafkaClientService kafkaClient;
 
   @BeforeClass
   public static void init() throws Exception {
@@ -72,12 +63,12 @@ public class KafkaTest {
     zkServer.startAndWait();
 
     // Extract the kafka.tgz and start the kafka server
-    kafkaServer = new EmbeddedKafkaServer(extractKafka(), generateKafkaConfig(zkServer.getConnectionStr()));
+    kafkaServer = new EmbeddedKafkaServer(generateKafkaConfig(zkServer.getConnectionStr()));
     kafkaServer.startAndWait();
 
     zkClientService = ZKClientService.Builder.of(zkServer.getConnectionStr()).build();
 
-    kafkaClient = new SimpleKafkaClient(zkClientService);
+    kafkaClient = new ZKKafkaClientService(zkClientService);
     Services.chainStart(zkClientService, kafkaClient).get();
   }
 
@@ -102,46 +93,27 @@ public class KafkaTest {
     t2.join();
     t3.start();
 
-    Iterator<FetchedMessage> consumer = kafkaClient.consume(topic, 0, 0, 1048576);
-    int count = 0;
-    long startTime = System.nanoTime();
-    while (count < 30 && consumer.hasNext() && secondsPassed(startTime, TimeUnit.NANOSECONDS) < 5) {
-      LOG.info(Charsets.UTF_8.decode(consumer.next().getBuffer()).toString());
-      count++;
-    }
-
-    Assert.assertEquals(30, count);
-  }
-
-  @Test (timeout = 10000)
-  public void testOffset() throws Exception {
-    String topic = "testOffset";
-
-    // Initial earliest offset should be 0.
-    long[] offsets = kafkaClient.getOffset(topic, 0, -2, 10).get();
-    Assert.assertArrayEquals(new long[]{0L}, offsets);
-
-    // Publish some messages
-    Thread publishThread = createPublishThread(kafkaClient, topic, Compression.NONE, "Testing", 2000);
-    publishThread.start();
-    publishThread.join();
-
-    // Fetch earliest offset, should still be 0.
-    offsets = kafkaClient.getOffset(topic, 0, -2, 10).get();
-    Assert.assertArrayEquals(new long[]{0L}, offsets);
-
-    // Fetch latest offset
-    offsets = kafkaClient.getOffset(topic, 0, -1, 10).get();
-    Iterator<FetchedMessage> consumer = kafkaClient.consume(topic, 0, offsets[0], 1048576);
+    final CountDownLatch latch = new CountDownLatch(30);
+    final CountDownLatch stopLatch = new CountDownLatch(1);
+    Cancellable cancel = kafkaClient.getConsumer().prepare().add(topic, 0, 0).consume(new KafkaConsumer
+      .MessageCallback() {
+      @Override
+      public void onReceived(Iterator<FetchedMessage> messages) {
+        while (messages.hasNext()) {
+          LOG.info(Charsets.UTF_8.decode(messages.next().getPayload()).toString());
+          latch.countDown();
+        }
+      }
 
-    // Publish one more message, the consumer should see the new message being published.
-    publishThread = createPublishThread(kafkaClient, topic, Compression.NONE, "Testing", 1, 3000);
-    publishThread.start();
-    publishThread.join();
+      @Override
+      public void finished() {
+        stopLatch.countDown();
+      }
+    });
 
-    // Should see the last message being published.
-    Assert.assertTrue(consumer.hasNext());
-    Assert.assertEquals("3000 Testing", Charsets.UTF_8.decode(consumer.next().getBuffer()).toString());
+    Assert.assertTrue(latch.await(5, TimeUnit.SECONDS));
+    cancel.cancel();
+    Assert.assertTrue(stopLatch.await(1, TimeUnit.SECONDS));
   }
 
   private Thread createPublishThread(final KafkaClient kafkaClient, final String topic,
@@ -153,11 +125,12 @@ public class KafkaTest {
                                      final String message, final int count, final int base) {
     return new Thread() {
       public void run() {
-        PreparePublish preparePublish = kafkaClient.preparePublish(topic, compression);
+        KafkaPublisher publisher = kafkaClient.getPublisher(KafkaPublisher.Ack.ALL_RECEIVED, compression);
+        KafkaPublisher.Preparer preparer = publisher.prepare(topic);
         for (int i = 0; i < count; i++) {
-          preparePublish.add(((base + i) + " " + message).getBytes(Charsets.UTF_8), 0);
+          preparer.add(Charsets.UTF_8.encode((base + i) + " " + message), 0);
         }
-        Futures.getUnchecked(preparePublish.publish());
+        Futures.getUnchecked(preparer.send());
       }
     };
   }
@@ -167,30 +140,6 @@ public class KafkaTest {
                                     TimeUnit.NANOSECONDS);
   }
 
-  private static File extractKafka() throws IOException, ArchiveException, CompressorException {
-    File kafkaExtract = TMP_FOLDER.newFolder();
-    InputStream kakfaResource = KafkaTest.class.getClassLoader().getResourceAsStream("kafka-0.7.2.tgz");
-    ArchiveInputStream archiveInput = new ArchiveStreamFactory()
-      .createArchiveInputStream(ArchiveStreamFactory.TAR,
-                                new CompressorStreamFactory()
-                                  .createCompressorInputStream(CompressorStreamFactory.GZIP, kakfaResource));
-
-    try {
-      ArchiveEntry entry = archiveInput.getNextEntry();
-      while (entry != null) {
-        File file = new File(kafkaExtract, entry.getName());
-        if (entry.isDirectory()) {
-          file.mkdirs();
-        } else {
-          ByteStreams.copy(archiveInput, Files.newOutputStreamSupplier(file));
-        }
-        entry = archiveInput.getNextEntry();
-      }
-    } finally {
-      archiveInput.close();
-    }
-    return kafkaExtract;
-  }
 
   private static Properties generateKafkaConfig(String zkConnectStr) throws IOException {
     int port = Networks.getRandomPort();
@@ -198,20 +147,20 @@ public class KafkaTest {
 
     Properties prop = new Properties();
     prop.setProperty("log.dir", TMP_FOLDER.newFolder().getAbsolutePath());
-    prop.setProperty("zk.connect", zkConnectStr);
-    prop.setProperty("num.threads", "8");
     prop.setProperty("port", Integer.toString(port));
-    prop.setProperty("log.flush.interval", "1000");
-    prop.setProperty("max.socket.request.bytes", "104857600");
-    prop.setProperty("log.cleanup.interval.mins", "1");
-    prop.setProperty("log.default.flush.scheduler.interval.ms", "1000");
-    prop.setProperty("zk.connectiontimeout.ms", "1000000");
-    prop.setProperty("socket.receive.buffer", "1048576");
-    prop.setProperty("enable.zookeeper", "true");
-    prop.setProperty("log.retention.hours", "24");
-    prop.setProperty("brokerid", "0");
-    prop.setProperty("socket.send.buffer", "1048576");
+    prop.setProperty("broker.id", "1");
+    prop.setProperty("socket.send.buffer.bytes", "1048576");
+    prop.setProperty("socket.receive.buffer.bytes", "1048576");
+    prop.setProperty("socket.request.max.bytes", "104857600");
     prop.setProperty("num.partitions", "1");
+    prop.setProperty("log.retention.hours", "1");
+    prop.setProperty("log.flush.interval.messages", "10000");
+    prop.setProperty("log.flush.interval.ms", "1000");
+    prop.setProperty("log.segment.bytes", "536870912");
+    prop.setProperty("zookeeper.connect", zkConnectStr);
+    prop.setProperty("zookeeper.connection.timeout.ms", "1000000");
+    prop.setProperty("default.replication.factor", "1");
+
     // Use a really small file size to force some flush to happen
     prop.setProperty("log.file.size", "1024");
     prop.setProperty("log.default.flush.interval.ms", "1000");

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
index 51c8503..73cecf0 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
@@ -17,6 +17,39 @@
  */
 package org.apache.twill.internal.appmaster;
 
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.base.Supplier;
+import com.google.common.base.Throwables;
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multiset;
+import com.google.common.collect.Sets;
+import com.google.common.io.CharStreams;
+import com.google.common.io.Files;
+import com.google.common.io.InputSupplier;
+import com.google.common.reflect.TypeToken;
+import com.google.common.util.concurrent.AbstractExecutionThreadService;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.Service;
+import com.google.common.util.concurrent.SettableFuture;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonElement;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.Records;
 import org.apache.twill.api.Command;
 import org.apache.twill.api.EventHandler;
 import org.apache.twill.api.EventHandlerSpecification;
@@ -51,39 +84,6 @@ import org.apache.twill.internal.yarn.YarnContainerStatus;
 import org.apache.twill.internal.yarn.YarnUtils;
 import org.apache.twill.zookeeper.ZKClient;
 import org.apache.twill.zookeeper.ZKClients;
-import com.google.common.base.Charsets;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import com.google.common.base.Supplier;
-import com.google.common.base.Throwables;
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableMultimap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multiset;
-import com.google.common.collect.Sets;
-import com.google.common.io.CharStreams;
-import com.google.common.io.Files;
-import com.google.common.io.InputSupplier;
-import com.google.common.reflect.TypeToken;
-import com.google.common.util.concurrent.AbstractExecutionThreadService;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.Service;
-import com.google.common.util.concurrent.SettableFuture;
-import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
-import com.google.gson.JsonElement;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.util.Records;
 import org.apache.zookeeper.CreateMode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -241,7 +241,7 @@ public final class ApplicationMasterService extends AbstractTwillService {
 
     instanceChangeExecutor = Executors.newSingleThreadExecutor(Threads.createDaemonThreadFactory("instanceChanger"));
 
-    kafkaServer = new EmbeddedKafkaServer(new File(Constants.Files.KAFKA), generateKafkaConfig());
+    kafkaServer = new EmbeddedKafkaServer(generateKafkaConfig());
 
     // Must start tracker before start AMClient
     LOG.info("Starting application master tracker server");
@@ -589,22 +589,19 @@ public final class ApplicationMasterService extends AbstractTwillService {
 
     Properties prop = new Properties();
     prop.setProperty("log.dir", new File("kafka-logs").getAbsolutePath());
-    prop.setProperty("zk.connect", getKafkaZKConnect());
-    prop.setProperty("num.threads", "8");
     prop.setProperty("port", Integer.toString(port));
-    prop.setProperty("log.flush.interval", "10000");
-    prop.setProperty("max.socket.request.bytes", "104857600");
-    prop.setProperty("log.cleanup.interval.mins", "1");
-    prop.setProperty("log.default.flush.scheduler.interval.ms", "1000");
-    prop.setProperty("zk.connectiontimeout.ms", "1000000");
-    prop.setProperty("socket.receive.buffer", "1048576");
-    prop.setProperty("enable.zookeeper", "true");
-    prop.setProperty("log.retention.hours", "24");
-    prop.setProperty("brokerid", "0");
-    prop.setProperty("socket.send.buffer", "1048576");
+    prop.setProperty("broker.id", "1");
+    prop.setProperty("socket.send.buffer.bytes", "1048576");
+    prop.setProperty("socket.receive.buffer.bytes", "1048576");
+    prop.setProperty("socket.request.max.bytes", "104857600");
     prop.setProperty("num.partitions", "1");
-    prop.setProperty("log.file.size", "536870912");
-    prop.setProperty("log.default.flush.interval.ms", "1000");
+    prop.setProperty("log.retention.hours", "24");
+    prop.setProperty("log.flush.interval.messages", "10000");
+    prop.setProperty("log.flush.interval.ms", "1000");
+    prop.setProperty("log.segment.bytes", "536870912");
+    prop.setProperty("zookeeper.connect", getKafkaZKConnect());
+    prop.setProperty("zookeeper.connection.timeout.ms", "1000000");
+    prop.setProperty("default.replication.factor", "1");
     return prop;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
index 17425d4..8c96629 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
@@ -243,7 +243,6 @@ final class YarnTwillPreparer implements TwillPreparer {
           saveSpecification(twillSpec, runnableLocalFiles, localFiles);
           saveLogback(localFiles);
           saveLauncher(localFiles);
-          saveKafka(localFiles);
           saveVmOptions(vmOpts, localFiles);
           saveArguments(new Arguments(arguments, runnableArgs), localFiles);
           saveLocalFiles(localFiles, ImmutableSet.of(Constants.Files.TWILL_SPEC,
@@ -498,15 +497,6 @@ final class YarnTwillPreparer implements TwillPreparer {
     localFiles.put(Constants.Files.LAUNCHER_JAR, createLocalFile(Constants.Files.LAUNCHER_JAR, location));
   }
 
-  private void saveKafka(Map<String, LocalFile> localFiles) throws IOException {
-    LOG.debug("Copy {}", Constants.Files.KAFKA);
-    Location location = copyFromURL(getClass().getClassLoader().getResource(KAFKA_ARCHIVE),
-                                    createTempLocation(Constants.Files.KAFKA));
-    LOG.debug("Done {}", Constants.Files.KAFKA);
-
-    localFiles.put(Constants.Files.KAFKA, createLocalFile(Constants.Files.KAFKA, location, true));
-  }
-
   private void saveVmOptions(String opts, Map<String, LocalFile> localFiles) throws IOException {
     if (opts.isEmpty()) {
       // If no vm options, no need to localize the file.


[17/50] [abbrv] git commit: [TWILL-37] Cleanup NOTICE file to remove license statements that are not needed.

Posted by ch...@apache.org.
[TWILL-37] Cleanup NOTICE file to remove license statements that are not needed.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 78eb1e1e25fae4d11fc71b10af77b70f5c4ee532
Parents: ed41c44
Author: Terence Yim <te...@continuuity.com>
Authored: Tue Jan 28 13:55:35 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Tue Jan 28 14:06:07 2014 -0800

----------------------------------------------------------------------
 NOTICE | 34 ++++------------------------------
 1 file changed, 4 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/78eb1e1e/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index df3ad5e..194805c 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,31 +1,5 @@
-This product includes software developed by The Apache Software
-Foundation (http://www.apache.org/).
+Apache Twill
+Copyright [2013-2014] The Apache Software Foundation
 
-In addition, this product includes software developed by:
-
-Guava (https://code.google.com/p/guava-libraries) is a Google Core Libraries.
-Licensed under the Apache License 2.0.
-
-Netty (http://netty.io) is a an asynchronous event-driven network application framework.
-Licensed under the Apache License 2.0.
-
-Findbugs (https://code.google.com/p/findbugs) is a defect detection tool for Java.
-Licensed under the GNU Lesser GPL
-
-Gson (https://code.google.com/p/google-gson) is a Java library for converting Java Objects into JSON.
-Licensed under the Apache License 2.0.
-
-Snappy-java (https://code.google.com/p/snappy-java) is a Java library for compression.
-Licensed under the Apache License 2.0.
-
-SLF4J (http://www.slf4j.org/) is a logging library for Java.
-Licensed under the MIT License.
-
-Logback (http://logback.qos.ch) is a logging library for Java.
-Dual licensed under EPL/LGPL. We use it under LGPL.
-
-ASM (http://asm.ow2.org) is a bytecode manipulation library for Java.
-Licensed under the BSD License.
-
-JUnit (http://www.junit.org/) included under the Common Public License v1.0. See
-the full text here: http://junit.sourceforge.net/cpl-v10.html
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).


[05/50] [abbrv] [TWILL-27] Upgrade to Kafka-0.8

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaClient.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaClient.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaClient.java
deleted file mode 100644
index 8ff4856..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaClient.java
+++ /dev/null
@@ -1,304 +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.twill.internal.kafka.client;
-
-import org.apache.twill.common.Threads;
-import org.apache.twill.kafka.client.FetchException;
-import org.apache.twill.kafka.client.FetchedMessage;
-import org.apache.twill.kafka.client.KafkaClient;
-import org.apache.twill.kafka.client.PreparePublish;
-import org.apache.twill.zookeeper.ZKClient;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.primitives.Ints;
-import com.google.common.primitives.Longs;
-import com.google.common.util.concurrent.AbstractIdleService;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.SettableFuture;
-import org.jboss.netty.bootstrap.ClientBootstrap;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelFutureListener;
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.ChannelPipelineFactory;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.socket.nio.NioClientBossPool;
-import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
-import org.jboss.netty.channel.socket.nio.NioWorkerPool;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.ByteBuffer;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * Basic implementation of {@link KafkaClient}.
- */
-public final class SimpleKafkaClient extends AbstractIdleService implements KafkaClient {
-
-  private static final Logger LOG = LoggerFactory.getLogger(SimpleKafkaClient.class);
-  private static final int BROKER_POLL_INTERVAL = 100;
-
-  private final KafkaBrokerCache brokerCache;
-  private ClientBootstrap bootstrap;
-  private ConnectionPool connectionPool;
-
-  public SimpleKafkaClient(ZKClient zkClient) {
-    this.brokerCache = new KafkaBrokerCache(zkClient);
-  }
-
-  @Override
-  protected void startUp() throws Exception {
-    brokerCache.startAndWait();
-    ThreadFactory threadFactory = Threads.createDaemonThreadFactory("kafka-client-netty-%d");
-    NioClientBossPool bossPool = new NioClientBossPool(Executors.newSingleThreadExecutor(threadFactory), 1,
-                                                       new HashedWheelTimer(threadFactory), null);
-    NioWorkerPool workerPool = new NioWorkerPool(Executors.newFixedThreadPool(4, threadFactory), 4);
-
-    bootstrap = new ClientBootstrap(new NioClientSocketChannelFactory(bossPool, workerPool));
-    bootstrap.setPipelineFactory(new KafkaChannelPipelineFactory());
-    connectionPool = new ConnectionPool(bootstrap);
-  }
-
-  @Override
-  protected void shutDown() throws Exception {
-    connectionPool.close();
-    bootstrap.releaseExternalResources();
-    brokerCache.stopAndWait();
-  }
-
-  @Override
-  public PreparePublish preparePublish(final String topic, final Compression compression) {
-    final Map<Integer, MessageSetEncoder> encoders = Maps.newHashMap();
-
-    return new PreparePublish() {
-      @Override
-      public PreparePublish add(byte[] payload, Object partitionKey) {
-        return add(ByteBuffer.wrap(payload), partitionKey);
-      }
-
-      @Override
-      public PreparePublish add(ByteBuffer payload, Object partitionKey) {
-        // TODO: Partition
-        int partition = 0;
-
-        MessageSetEncoder encoder = encoders.get(partition);
-        if (encoder == null) {
-          encoder = getEncoder(compression);
-          encoders.put(partition, encoder);
-        }
-        encoder.add(ChannelBuffers.wrappedBuffer(payload));
-
-        return this;
-      }
-
-      @Override
-      public ListenableFuture<?> publish() {
-        List<ListenableFuture<?>> futures = Lists.newArrayListWithCapacity(encoders.size());
-        for (Map.Entry<Integer, MessageSetEncoder> entry : encoders.entrySet()) {
-          futures.add(doPublish(topic, entry.getKey(), entry.getValue().finish()));
-        }
-        encoders.clear();
-        return Futures.allAsList(futures);
-      }
-
-      private ListenableFuture<?> doPublish(String topic, int partition, ChannelBuffer messageSet) {
-        final KafkaRequest request = KafkaRequest.createProduce(topic, partition, messageSet);
-        final SettableFuture<?> result = SettableFuture.create();
-        final ConnectionPool.ConnectResult connection =
-              connectionPool.connect(getTopicBroker(topic, partition).getAddress());
-
-        connection.getChannelFuture().addListener(new ChannelFutureListener() {
-          @Override
-          public void operationComplete(ChannelFuture future) throws Exception {
-            try {
-              future.getChannel().write(request).addListener(getPublishChannelFutureListener(result, null, connection));
-            } catch (Exception e) {
-              result.setException(e);
-            }
-          }
-        });
-
-        return result;
-      }
-    };
-  }
-
-  @Override
-  public Iterator<FetchedMessage> consume(final String topic, final int partition, long offset, int maxSize) {
-    Preconditions.checkArgument(maxSize >= 10, "Message size cannot be smaller than 10.");
-
-    // Connect to broker. Consumer connection are long connection. No need to worry about reuse.
-    final AtomicReference<ChannelFuture> channelFutureRef = new AtomicReference<ChannelFuture>(
-          connectionPool.connect(getTopicBroker(topic, partition).getAddress()).getChannelFuture());
-
-    return new MessageFetcher(topic, partition, offset, maxSize, new KafkaRequestSender() {
-
-      @Override
-      public void send(final KafkaRequest request) {
-        if (!isRunning()) {
-          return;
-        }
-        try {
-          // Try to send the request
-          Channel channel = channelFutureRef.get().getChannel();
-          if (!channel.write(request).await().isSuccess()) {
-            // If failed, retry
-            channel.close();
-            ChannelFuture channelFuture = connectionPool.connect(
-                                              getTopicBroker(topic, partition).getAddress()).getChannelFuture();
-            channelFutureRef.set(channelFuture);
-            channelFuture.addListener(new ChannelFutureListener() {
-              @Override
-              public void operationComplete(ChannelFuture channelFuture) throws Exception {
-                send(request);
-              }
-            });
-          }
-        } catch (InterruptedException e) {
-          // Ignore it
-          LOG.info("Interrupted when sending consume request", e);
-        }
-      }
-    });
-  }
-
-  @Override
-  public ListenableFuture<long[]> getOffset(final String topic, final int partition, long time, int maxOffsets) {
-    final SettableFuture<long[]> resultFuture = SettableFuture.create();
-    final ChannelBuffer body = ChannelBuffers.buffer(Longs.BYTES + Ints.BYTES);
-    body.writeLong(time);
-    body.writeInt(maxOffsets);
-
-    connectionPool.connect(getTopicBroker(topic, partition).getAddress())
-                  .getChannelFuture().addListener(new ChannelFutureListener() {
-      @Override
-      public void operationComplete(ChannelFuture future) throws Exception {
-        if (checkFailure(future)) {
-          return;
-        }
-
-        future.getChannel().write(KafkaRequest.createOffsets(topic, partition, body, new ResponseHandler() {
-          @Override
-          public void received(KafkaResponse response) {
-            if (response.getErrorCode() != FetchException.ErrorCode.OK) {
-              resultFuture.setException(new FetchException("Failed to fetch offset.", response.getErrorCode()));
-            } else {
-              // Decode the offset response, which contains 4 bytes number of offsets, followed by number of offsets,
-              // each 8 bytes in size.
-              ChannelBuffer resultBuffer = response.getBody();
-              int size = resultBuffer.readInt();
-              long[] result = new long[size];
-              for (int i = 0; i < size; i++) {
-                result[i] = resultBuffer.readLong();
-              }
-              resultFuture.set(result);
-            }
-          }
-        })).addListener(new ChannelFutureListener() {
-          @Override
-          public void operationComplete(ChannelFuture future) throws Exception {
-            checkFailure(future);
-          }
-        });
-      }
-
-      private boolean checkFailure(ChannelFuture future) {
-        if (!future.isSuccess()) {
-          if (future.isCancelled()) {
-            resultFuture.cancel(true);
-          } else {
-            resultFuture.setException(future.getCause());
-          }
-          return true;
-        }
-        return false;
-      }
-    });
-
-    return resultFuture;
-  }
-
-  private TopicBroker getTopicBroker(String topic, int partition) {
-    TopicBroker topicBroker = brokerCache.getBrokerAddress(topic, partition);
-    while (topicBroker == null) {
-      try {
-        TimeUnit.MILLISECONDS.sleep(BROKER_POLL_INTERVAL);
-      } catch (InterruptedException e) {
-        return null;
-      }
-      topicBroker = brokerCache.getBrokerAddress(topic, partition);
-    }
-    return topicBroker;
-  }
-
-  private MessageSetEncoder getEncoder(Compression compression) {
-    switch (compression) {
-      case GZIP:
-        return new GZipMessageSetEncoder();
-      case SNAPPY:
-        return new SnappyMessageSetEncoder();
-      default:
-        return new IdentityMessageSetEncoder();
-    }
-  }
-
-  private <V> ChannelFutureListener getPublishChannelFutureListener(final SettableFuture<V> result, final V resultObj,
-                                                                    final ConnectionPool.ConnectionReleaser releaser) {
-    return new ChannelFutureListener() {
-      @Override
-      public void operationComplete(ChannelFuture future) throws Exception {
-        try {
-          if (future.isSuccess()) {
-            result.set(resultObj);
-          } else if (future.isCancelled()) {
-            result.cancel(true);
-          } else {
-            result.setException(future.getCause());
-          }
-        } finally {
-          releaser.release();
-        }
-      }
-    };
-  }
-
-  private static final class KafkaChannelPipelineFactory implements ChannelPipelineFactory {
-
-    @Override
-    public ChannelPipeline getPipeline() throws Exception {
-      ChannelPipeline pipeline = Channels.pipeline();
-
-      pipeline.addLast("encoder", new KafkaRequestEncoder());
-      pipeline.addLast("decoder", new KafkaResponseHandler());
-      pipeline.addLast("dispatcher", new KafkaResponseDispatcher());
-      return pipeline;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaConsumer.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaConsumer.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaConsumer.java
new file mode 100644
index 0000000..230521c
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaConsumer.java
@@ -0,0 +1,481 @@
+/*
+ * 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.twill.internal.kafka.client;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.Futures;
+import kafka.api.FetchRequest;
+import kafka.api.FetchRequestBuilder;
+import kafka.api.PartitionOffsetRequestInfo;
+import kafka.common.ErrorMapping;
+import kafka.common.TopicAndPartition;
+import kafka.javaapi.FetchResponse;
+import kafka.javaapi.OffsetRequest;
+import kafka.javaapi.OffsetResponse;
+import kafka.javaapi.consumer.SimpleConsumer;
+import kafka.javaapi.message.ByteBufferMessageSet;
+import kafka.message.MessageAndOffset;
+import org.apache.twill.common.Cancellable;
+import org.apache.twill.common.Threads;
+import org.apache.twill.kafka.client.BrokerInfo;
+import org.apache.twill.kafka.client.BrokerService;
+import org.apache.twill.kafka.client.FetchedMessage;
+import org.apache.twill.kafka.client.KafkaConsumer;
+import org.apache.twill.kafka.client.TopicPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A {@link KafkaConsumer} implementation using the scala kafka api.
+ */
+final class SimpleKafkaConsumer implements KafkaConsumer {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SimpleKafkaConsumer.class);
+  private static final int FETCH_SIZE = 1024 * 1024;        // Use a default fetch size.
+  private static final int SO_TIMEOUT = 5 * 1000;           // 5 seconds.
+  private static final int MAX_WAIT = 1000;                 // 1 second.
+  private static final long CONSUMER_EXPIRE_MINUTES = 1L;   // close consumer if not used for 1 minute.
+  private static final long CONSUMER_FAILURE_RETRY_INTERVAL = 2000L; // Sleep for 2 seconds if failure in consumer.
+  private static final long EMPTY_FETCH_WAIT = 500L;        // Sleep for 500 ms if no message is fetched.
+
+  private final BrokerService brokerService;
+  private final LoadingCache<BrokerInfo, SimpleConsumer> consumers;
+  private final BlockingQueue<Cancellable> consumerCancels;
+
+  SimpleKafkaConsumer(BrokerService brokerService) {
+    this.brokerService = brokerService;
+    this.consumers = CacheBuilder.newBuilder()
+                                 .expireAfterAccess(CONSUMER_EXPIRE_MINUTES, TimeUnit.MINUTES)
+                                 .removalListener(createRemovalListener())
+                                 .build(createConsumerLoader());
+    this.consumerCancels = new LinkedBlockingQueue<Cancellable>();
+  }
+
+  @Override
+  public Preparer prepare() {
+    return new SimplePreparer();
+  }
+
+  /**
+   * Called to stop all consumers created. This method should only be
+   * called by KafkaClientService who own this consumer.
+   */
+  void stop() {
+    LOG.info("Stopping Kafka consumer");
+    List<Cancellable> cancels = Lists.newLinkedList();
+    consumerCancels.drainTo(cancels);
+    for (Cancellable cancel : cancels) {
+      cancel.cancel();
+    }
+    consumers.invalidateAll();
+    LOG.info("Kafka Consumer stopped");
+  }
+
+  /**
+   * Creates a CacheLoader for creating SimpleConsumer.
+   */
+  private CacheLoader<BrokerInfo, SimpleConsumer> createConsumerLoader() {
+    return new CacheLoader<BrokerInfo, SimpleConsumer>() {
+      @Override
+      public SimpleConsumer load(BrokerInfo key) throws Exception {
+        return new SimpleConsumer(key.getHost(), key.getPort(), SO_TIMEOUT, FETCH_SIZE, "simple-kafka-client");
+      }
+    };
+  }
+
+  /**
+   * Creates a RemovalListener that will close SimpleConsumer on cache removal.
+   */
+  private RemovalListener<BrokerInfo, SimpleConsumer> createRemovalListener() {
+    return new RemovalListener<BrokerInfo, SimpleConsumer>() {
+      @Override
+      public void onRemoval(RemovalNotification<BrokerInfo, SimpleConsumer> notification) {
+        SimpleConsumer consumer = notification.getValue();
+        if (consumer != null) {
+          consumer.close();
+        }
+      }
+    };
+  }
+
+  /**
+   * Retrieves the last offset before the given timestamp for a given topic partition.
+   *
+   * @return The last offset before the given timestamp or {@code 0} if failed to do so.
+   */
+  private long getLastOffset(TopicPartition topicPart, long timestamp) {
+    BrokerInfo brokerInfo = brokerService.getLeader(topicPart.getTopic(), topicPart.getPartition());
+    SimpleConsumer consumer = brokerInfo == null ? null : consumers.getUnchecked(brokerInfo);
+
+    // If no broker, treat it as failure attempt.
+    if (consumer == null) {
+      LOG.warn("Failed to talk to any broker. Default offset to 0 for {}", topicPart);
+      return 0L;
+    }
+
+    // Fire offset request
+    OffsetRequest request = new OffsetRequest(ImmutableMap.of(
+      new TopicAndPartition(topicPart.getTopic(), topicPart.getPartition()),
+      new PartitionOffsetRequestInfo(timestamp, 1)
+    ), kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId());
+
+    OffsetResponse response = consumer.getOffsetsBefore(request);
+
+    // Retrieve offsets from response
+    long[] offsets = response.hasError() ? null : response.offsets(topicPart.getTopic(), topicPart.getPartition());
+    if (offsets == null || offsets.length <= 0) {
+      short errorCode = response.errorCode(topicPart.getTopic(), topicPart.getPartition());
+
+      // If the topic partition doesn't exists, use offset 0 without logging error.
+      if (errorCode != ErrorMapping.UnknownTopicOrPartitionCode()) {
+        consumers.refresh(brokerInfo);
+        LOG.warn("Failed to fetch offset for {} with timestamp {}. Error: {}. Default offset to 0.",
+                 topicPart, timestamp, errorCode);
+      }
+      return 0L;
+    }
+
+    LOG.debug("Offset {} fetched for {} with timestamp {}.", offsets[0], topicPart, timestamp);
+    return offsets[0];
+  }
+
+  /**
+   * A preparer that uses kafak scala api for consuming messages.
+   */
+  private final class SimplePreparer implements Preparer {
+
+    // Map from TopicPartition to offset
+    private final Map<TopicPartition, Long> requests;
+    private final ThreadFactory threadFactory;
+
+    private SimplePreparer() {
+      this.requests = Maps.newHashMap();
+      this.threadFactory = Threads.createDaemonThreadFactory("message-callback-%d");
+    }
+
+    @Override
+    public Preparer add(String topic, int partition, long offset) {
+      requests.put(new TopicPartition(topic, partition), offset);
+      return this;
+    }
+
+    @Override
+    public Preparer addFromBeginning(String topic, int partition) {
+      TopicPartition topicPartition = new TopicPartition(topic, partition);
+      requests.put(topicPartition, kafka.api.OffsetRequest.EarliestTime());
+      return this;
+    }
+
+    @Override
+    public Preparer addLatest(String topic, int partition) {
+      TopicPartition topicPartition = new TopicPartition(topic, partition);
+      requests.put(topicPartition, kafka.api.OffsetRequest.LatestTime());
+      return this;
+    }
+
+    @Override
+    public Cancellable consume(MessageCallback callback) {
+      final ExecutorService executor = Executors.newSingleThreadExecutor(threadFactory);
+      final List<ConsumerThread> pollers = Lists.newArrayList();
+
+      // When cancelling the consumption, first terminates all polling threads and then stop the executor service.
+      final AtomicBoolean cancelled = new AtomicBoolean();
+      Cancellable cancellable = new Cancellable() {
+        @Override
+        public void cancel() {
+          if (!cancelled.compareAndSet(false, true)) {
+            return;
+          }
+          consumerCancels.remove(this);
+
+          LOG.info("Requesting stop of all consumer threads.");
+          for (ConsumerThread consumerThread : pollers) {
+            consumerThread.terminate();
+          }
+          LOG.info("Wait for all consumer threads to stop.");
+          for (ConsumerThread consumerThread : pollers) {
+            try {
+              consumerThread.join();
+            } catch (InterruptedException e) {
+              LOG.warn("Interrupted exception while waiting for thread to complete.", e);
+            }
+          }
+          LOG.info("All consumer threads stopped.");
+          // Use shutdown so that submitted task still has chance to execute, which is important for finished to get
+          // called.
+          executor.shutdown();
+        }
+      };
+
+      // Wrap the callback with a single thread executor.
+      MessageCallback messageCallback = wrapCallback(callback, executor, cancellable);
+
+      // Starts threads for polling new messages.
+      for (Map.Entry<TopicPartition, Long> entry : requests.entrySet()) {
+        ConsumerThread consumerThread = new ConsumerThread(entry.getKey(), entry.getValue(), messageCallback);
+        consumerThread.setDaemon(true);
+        consumerThread.start();
+        pollers.add(consumerThread);
+      }
+
+      consumerCancels.add(cancellable);
+      return cancellable;
+    }
+
+    /**
+     * Wrap a given MessageCallback by a executor so that calls are executed in the given executor.
+     * By running the calls through the executor, it also block and wait for the task being completed so that
+     * it can block the poller thread depending on the rate of processing that the callback can handle.
+     */
+    private MessageCallback wrapCallback(final MessageCallback callback,
+                                         final ExecutorService executor, final Cancellable cancellable) {
+      final AtomicBoolean stopped = new AtomicBoolean();
+      return new MessageCallback() {
+        @Override
+        public void onReceived(final Iterator<FetchedMessage> messages) {
+          if (stopped.get()) {
+            return;
+          }
+          Futures.getUnchecked(executor.submit(new Runnable() {
+            @Override
+            public void run() {
+              if (stopped.get()) {
+                return;
+              }
+              callback.onReceived(messages);
+            }
+          }));
+        }
+
+        @Override
+        public void finished() {
+          // Make sure finished only get called once.
+          if (!stopped.compareAndSet(false, true)) {
+            return;
+          }
+          Futures.getUnchecked(executor.submit(new Runnable() {
+            @Override
+            public void run() {
+              // When finished is called, also cancel the consumption from all polling thread.
+              callback.finished();
+              cancellable.cancel();
+            }
+          }));
+        }
+      };
+    }
+  }
+
+  /**
+   * The thread for polling kafka.
+   */
+  private final class ConsumerThread extends Thread {
+
+    private final TopicPartition topicPart;
+    private final long startOffset;
+    private final MessageCallback callback;
+    private final BasicFetchedMessage fetchedMessage;
+    private volatile boolean running;
+
+    private ConsumerThread(TopicPartition topicPart, long startOffset, MessageCallback callback) {
+      super(String.format("Kafka-Consumer-%s-%d", topicPart.getTopic(), topicPart.getPartition()));
+      this.topicPart = topicPart;
+      this.startOffset = startOffset;
+      this.callback = callback;
+      this.running = true;
+      this.fetchedMessage = new BasicFetchedMessage(topicPart);
+    }
+
+    @Override
+    public void run() {
+      final AtomicLong offset = new AtomicLong(startOffset);
+
+      Map.Entry<BrokerInfo, SimpleConsumer> consumerEntry = null;
+
+      while (running) {
+        if (consumerEntry == null && (consumerEntry = getConsumerEntry()) == null) {
+          LOG.debug("No leader for topic partition {}.", topicPart);
+          try {
+            TimeUnit.MILLISECONDS.sleep(CONSUMER_FAILURE_RETRY_INTERVAL);
+          } catch (InterruptedException e) {
+            // OK to ignore this, as interrupt would be caused by thread termination.
+            LOG.trace("Consumer sleep interrupted.", e);
+          }
+          continue;
+        }
+
+        // If offset < 0, meaning it's special offset value that needs to fetch either the earliest or latest offset
+        // from kafak server.
+        long off = offset.get();
+        if (off < 0) {
+          offset.set(getLastOffset(topicPart, off));
+        }
+
+        SimpleConsumer consumer = consumerEntry.getValue();
+
+        // Fire a fetch message request
+        try {
+          FetchResponse response = fetchMessages(consumer, offset.get());
+
+          // Failure response, set consumer entry to null and let next round of loop to handle it.
+          if (response.hasError()) {
+            short errorCode = response.errorCode(topicPart.getTopic(), topicPart.getPartition());
+            LOG.info("Failed to fetch message on {}. Error: {}", topicPart, errorCode);
+            // If it is out of range error, reset to earliest offset
+            if (errorCode == ErrorMapping.OffsetOutOfRangeCode()) {
+              offset.set(kafka.api.OffsetRequest.EarliestTime());
+            }
+
+            consumers.refresh(consumerEntry.getKey());
+            consumerEntry = null;
+            continue;
+          }
+
+          ByteBufferMessageSet messages = response.messageSet(topicPart.getTopic(), topicPart.getPartition());
+          if (sleepIfEmpty(messages)) {
+            continue;
+          }
+
+          // Call the callback
+          invokeCallback(messages, offset);
+        } catch (Throwable t) {
+          if (running || !(t instanceof ClosedByInterruptException)) {
+            // Only log if it is still running, otherwise, it just the interrupt caused by the stop.
+            LOG.info("Exception when fetching message on {}.", topicPart, t);
+          }
+          consumers.refresh(consumerEntry.getKey());
+          consumerEntry = null;
+        }
+      }
+
+      // When the thread is done, call the callback finished method.
+      try {
+        callback.finished();
+      } catch (Throwable t) {
+        LOG.error("Exception thrown from MessageCallback.finished({})", running, t);
+      }
+    }
+
+    public void terminate() {
+      LOG.info("Terminate requested {}", getName());
+      running = false;
+      interrupt();
+    }
+
+    /**
+     * Gets the leader broker and the associated SimpleConsumer for the current topic and partition.
+     */
+    private Map.Entry<BrokerInfo, SimpleConsumer> getConsumerEntry() {
+      BrokerInfo leader = brokerService.getLeader(topicPart.getTopic(), topicPart.getPartition());
+      return leader == null ? null : Maps.immutableEntry(leader, consumers.getUnchecked(leader));
+    }
+
+    /**
+     * Makes a call to kafka to fetch messages.
+     */
+    private FetchResponse fetchMessages(SimpleConsumer consumer, long offset) {
+      FetchRequest request = new FetchRequestBuilder()
+        .clientId(consumer.clientId())
+        .addFetch(topicPart.getTopic(), topicPart.getPartition(), offset, FETCH_SIZE)
+        .maxWait(MAX_WAIT)
+        .build();
+      return consumer.fetch(request);
+    }
+
+    /**
+     * Sleeps if the message set is empty.
+     * @return {@code true} if it is empty, {@code false} otherwise.
+     */
+    private boolean sleepIfEmpty(ByteBufferMessageSet messages) {
+      if (Iterables.isEmpty(messages)) {
+        LOG.trace("No message fetched. Sleep for {} ms before next fetch.", EMPTY_FETCH_WAIT);
+        try {
+          TimeUnit.MILLISECONDS.sleep(EMPTY_FETCH_WAIT);
+        } catch (InterruptedException e) {
+          // It's interrupted from stop, ok to ignore.
+        }
+        return true;
+      }
+      return false;
+    }
+
+    /**
+     * Calls the message callback with the given message set.
+     */
+    private void invokeCallback(ByteBufferMessageSet messages, AtomicLong offset) {
+      long savedOffset = offset.get();
+      try {
+        callback.onReceived(createFetchedMessages(messages, offset));
+      } catch (Throwable t) {
+        LOG.error("Callback throws exception. Retry from offset {} for {}", startOffset, topicPart, t);
+        offset.set(savedOffset);
+      }
+    }
+
+    /**
+     * Creates an Iterator of FetchedMessage based on the given message set. The iterator would also updates
+     * the offset while iterating.
+     */
+    private Iterator<FetchedMessage> createFetchedMessages(ByteBufferMessageSet messageSet, final AtomicLong offset) {
+      final Iterator<MessageAndOffset> messages = messageSet.iterator();
+      return new AbstractIterator<FetchedMessage>() {
+        @Override
+        protected FetchedMessage computeNext() {
+          while (messages.hasNext()) {
+            MessageAndOffset message = messages.next();
+            long msgOffset = message.offset();
+            if (msgOffset < offset.get()) {
+              LOG.trace("Received old offset {}, expecting {} on {}. Message Ignored.",
+                        msgOffset, offset.get(), topicPart);
+              continue;
+            }
+
+            offset.set(message.nextOffset());
+            fetchedMessage.setPayload(message.message().payload());
+            fetchedMessage.setNextOffset(offset.get());
+
+            return fetchedMessage;
+          }
+          return endOfData();
+        }
+      };
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaPublisher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaPublisher.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaPublisher.java
new file mode 100644
index 0000000..97b14c6
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaPublisher.java
@@ -0,0 +1,113 @@
+/*
+ * 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.twill.internal.kafka.client;
+
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import kafka.javaapi.producer.Producer;
+import kafka.producer.KeyedMessage;
+import kafka.producer.ProducerConfig;
+import org.apache.twill.common.Cancellable;
+import org.apache.twill.kafka.client.Compression;
+import org.apache.twill.kafka.client.KafkaPublisher;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Implementation of {@link KafkaPublisher} using the kafka scala-java api.
+ */
+final class SimpleKafkaPublisher implements KafkaPublisher {
+
+  private final String kafkaBrokers;
+  private final Ack ack;
+  private final Compression compression;
+  private Producer<Integer, ByteBuffer> producer;
+
+  public SimpleKafkaPublisher(String kafkaBrokers, Ack ack, Compression compression) {
+    this.kafkaBrokers = kafkaBrokers;
+    this.ack = ack;
+    this.compression = compression;
+  }
+
+  /**
+   * Start the publisher. This method must be called before other methods. This method is only to be called
+   * by KafkaClientService who own this object.
+   * @return A Cancellable for closing this publish.
+   */
+  Cancellable start() {
+    // It should return a Cancellable that is not holding any reference to this instance.
+    // It is for ZKKafkaClientService to close the internal Kafka producer when this publisher get garbage collected.
+    Properties props = new Properties();
+    props.put("metadata.broker.list", kafkaBrokers);
+    props.put("serializer.class", ByteBufferEncoder.class.getName());
+    props.put("key.serializer.class", IntegerEncoder.class.getName());
+    props.put("partitioner.class", IntegerPartitioner.class.getName());
+    props.put("request.required.acks", Integer.toString(ack.getAck()));
+    props.put("compression.codec", compression.getCodec());
+
+    producer = new Producer<Integer, ByteBuffer>(new ProducerConfig(props));
+    return new ProducerCancellable(producer);
+  }
+
+  @Override
+  public Preparer prepare(String topic) {
+    return new SimplePreparer(topic);
+  }
+
+  private final class SimplePreparer implements Preparer {
+
+    private final String topic;
+    private final List<KeyedMessage<Integer, ByteBuffer>> messages;
+
+    private SimplePreparer(String topic) {
+      this.topic = topic;
+      this.messages = Lists.newLinkedList();
+    }
+
+    @Override
+    public Preparer add(ByteBuffer message, Object partitionKey) {
+      messages.add(new KeyedMessage<Integer, ByteBuffer>(topic, Math.abs(partitionKey.hashCode()), message));
+      return this;
+    }
+
+    @Override
+    public ListenableFuture<Integer> send() {
+      int size = messages.size();
+      producer.send(messages);
+
+      messages.clear();
+      return Futures.immediateFuture(size);
+    }
+  }
+
+  private static final class ProducerCancellable implements Cancellable {
+    private final Producer<Integer, ByteBuffer> producer;
+
+    private ProducerCancellable(Producer<Integer, ByteBuffer> producer) {
+      this.producer = producer;
+    }
+
+    @Override
+    public void cancel() {
+      producer.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SnappyMessageSetEncoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SnappyMessageSetEncoder.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SnappyMessageSetEncoder.java
deleted file mode 100644
index bf18c08..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SnappyMessageSetEncoder.java
+++ /dev/null
@@ -1,38 +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.twill.internal.kafka.client;
-
-import org.xerial.snappy.SnappyOutputStream;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * A {@link MessageSetEncoder} that compress messages using snappy.
- */
-final class SnappyMessageSetEncoder extends AbstractCompressedMessageSetEncoder {
-
-  SnappyMessageSetEncoder() {
-    super(Compression.SNAPPY);
-  }
-
-  @Override
-  protected OutputStream createCompressedStream(OutputStream os) throws IOException {
-    return new SnappyOutputStream(os);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/TopicBroker.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/TopicBroker.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/TopicBroker.java
deleted file mode 100644
index fd4bf03..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/TopicBroker.java
+++ /dev/null
@@ -1,48 +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.twill.internal.kafka.client;
-
-import java.net.InetSocketAddress;
-
-/**
- * Represents broker information of a given topic.
- */
-final class TopicBroker {
-
-  private final String topic;
-  private final InetSocketAddress address;
-  private final int partitionSize;
-
-  TopicBroker(String topic, InetSocketAddress address, int partitionSize) {
-    this.topic = topic;
-    this.address = address;
-    this.partitionSize = partitionSize;
-  }
-
-  String getTopic() {
-    return topic;
-  }
-
-  InetSocketAddress getAddress() {
-    return address;
-  }
-
-  int getPartitionSize() {
-    return partitionSize;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ZKBrokerService.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ZKBrokerService.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ZKBrokerService.java
new file mode 100644
index 0000000..74ecfc1
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ZKBrokerService.java
@@ -0,0 +1,403 @@
+/*
+ * 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.twill.internal.kafka.client;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
+import com.google.common.base.Throwables;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.primitives.Ints;
+import com.google.common.util.concurrent.AbstractIdleService;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.SettableFuture;
+import com.google.gson.Gson;
+import org.apache.twill.common.Threads;
+import org.apache.twill.kafka.client.BrokerInfo;
+import org.apache.twill.kafka.client.BrokerService;
+import org.apache.twill.kafka.client.TopicPartition;
+import org.apache.twill.zookeeper.NodeChildren;
+import org.apache.twill.zookeeper.NodeData;
+import org.apache.twill.zookeeper.ZKClient;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A {@link BrokerService} that watches kafka zk nodes for updates of broker lists and leader for
+ * each topic partition.
+ */
+final class ZKBrokerService extends AbstractIdleService implements BrokerService {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ZKBrokerService.class);
+  private static final String BROKER_IDS_PATH = "/brokers/ids";
+  private static final String BROKER_TOPICS_PATH = "/brokers/topics";
+  private static final long FAILURE_RETRY_SECONDS = 5;
+  private static final Gson GSON = new Gson();
+  private static final Function<String, BrokerId> BROKER_ID_TRANSFORMER = new Function<String, BrokerId>() {
+    @Override
+    public BrokerId apply(String input) {
+      return new BrokerId(Integer.parseInt(input));
+    }
+  };
+  private static final Function<BrokerInfo, String> BROKER_INFO_TO_ADDRESS = new Function<BrokerInfo, String>() {
+    @Override
+    public String apply(BrokerInfo input) {
+      return String.format("%s:%d", input.getHost(), input.getPort());
+    }
+  };
+
+  private final ZKClient zkClient;
+  private final LoadingCache<BrokerId, Supplier<BrokerInfo>> brokerInfos;
+  private final LoadingCache<KeyPathTopicPartition, Supplier<PartitionInfo>> partitionInfos;
+  // A comma separated list of brokers (host:port,host:port)
+
+  private ExecutorService executorService;
+  private Supplier<Iterable<BrokerInfo>> brokerList;
+
+  ZKBrokerService(ZKClient zkClient) {
+    this.zkClient = zkClient;
+    this.brokerInfos = CacheBuilder.newBuilder().build(createCacheLoader(new CacheInvalidater<BrokerId>() {
+      @Override
+      public void invalidate(BrokerId key) {
+        brokerInfos.invalidate(key);
+      }
+    }, BrokerInfo.class));
+    this.partitionInfos = CacheBuilder.newBuilder().build(createCacheLoader(
+      new CacheInvalidater<KeyPathTopicPartition>() {
+      @Override
+      public void invalidate(KeyPathTopicPartition key) {
+        partitionInfos.invalidate(key);
+      }
+    }, PartitionInfo.class));
+  }
+
+  @Override
+  protected void startUp() throws Exception {
+    executorService = Executors.newCachedThreadPool(Threads.createDaemonThreadFactory("zk-kafka-broker"));
+  }
+
+  @Override
+  protected void shutDown() throws Exception {
+    executorService.shutdownNow();
+  }
+
+  @Override
+  public BrokerInfo getLeader(String topic, int partition) {
+    Preconditions.checkState(isRunning(), "BrokerService is not running.");
+    PartitionInfo partitionInfo = partitionInfos.getUnchecked(new KeyPathTopicPartition(topic, partition)).get();
+    return partitionInfo == null ? null : brokerInfos.getUnchecked(new BrokerId(partitionInfo.getLeader())).get();
+  }
+
+  @Override
+  public synchronized Iterable<BrokerInfo> getBrokers() {
+    Preconditions.checkState(isRunning(), "BrokerService is not running.");
+
+    if (brokerList != null) {
+      return brokerList.get();
+    }
+
+    final SettableFuture<?> readerFuture = SettableFuture.create();
+    final AtomicReference<Iterable<BrokerInfo>> brokers =
+      new AtomicReference<Iterable<BrokerInfo>>(ImmutableList.<BrokerInfo>of());
+
+    actOnExists(BROKER_IDS_PATH, new Runnable() {
+      @Override
+      public void run() {
+        // Callback for fetching children list. This callback should be executed in the executorService.
+        final FutureCallback<NodeChildren> childrenCallback = new FutureCallback<NodeChildren>() {
+          @Override
+          public void onSuccess(NodeChildren result) {
+            try {
+              // For each children node, get the BrokerInfo from the brokerInfo cache.
+              brokers.set(
+                ImmutableList.copyOf(
+                  Iterables.transform(
+                    brokerInfos.getAll(Iterables.transform(result.getChildren(), BROKER_ID_TRANSFORMER)).values(),
+                    Suppliers.<BrokerInfo>supplierFunction())));
+              readerFuture.set(null);
+
+            } catch (ExecutionException e) {
+              readerFuture.setException(e.getCause());
+            }
+          }
+
+          @Override
+          public void onFailure(Throwable t) {
+            readerFuture.setException(t);
+          }
+        };
+
+        // Fetch list of broker ids
+        Futures.addCallback(zkClient.getChildren(BROKER_IDS_PATH, new Watcher() {
+          @Override
+          public void process(WatchedEvent event) {
+            if (!isRunning()) {
+              return;
+            }
+            if (event.getType() == Event.EventType.NodeChildrenChanged) {
+              Futures.addCallback(zkClient.getChildren(BROKER_IDS_PATH, this), childrenCallback, executorService);
+            }
+          }
+        }), childrenCallback, executorService);
+      }
+    }, readerFuture, FAILURE_RETRY_SECONDS, TimeUnit.SECONDS);
+
+    brokerList = createSupplier(brokers);
+    try {
+      readerFuture.get();
+    } catch (Exception e) {
+      throw Throwables.propagate(e);
+    }
+    return brokerList.get();
+  }
+
+  @Override
+  public String getBrokerList() {
+    return Joiner.on(',').join(Iterables.transform(getBrokers(), BROKER_INFO_TO_ADDRESS));
+  }
+
+  /**
+   * Creates a cache loader for the given path to supply data with the data node.
+   */
+  private <K extends KeyPath, T> CacheLoader<K, Supplier<T>> createCacheLoader(final CacheInvalidater<K> invalidater,
+                                                                               final Class<T> resultType) {
+    return new CacheLoader<K, Supplier<T>>() {
+
+      @Override
+      public Supplier<T> load(final K key) throws Exception {
+        // A future to tell if the result is ready, even it is failure.
+        final SettableFuture<T> readyFuture = SettableFuture.create();
+        final AtomicReference<T> resultValue = new AtomicReference<T>();
+
+        // Fetch for node data when it exists.
+        final String path = key.getPath();
+        actOnExists(path, new Runnable() {
+          @Override
+          public void run() {
+            // Callback for getData call
+            final FutureCallback<NodeData> dataCallback = new FutureCallback<NodeData>() {
+              @Override
+              public void onSuccess(NodeData result) {
+                // Update with latest data
+                T value = decodeNodeData(result, resultType);
+                resultValue.set(value);
+                readyFuture.set(value);
+              }
+
+              @Override
+              public void onFailure(Throwable t) {
+                LOG.error("Failed to fetch node data on {}", path, t);
+                if (t instanceof KeeperException.NoNodeException) {
+                  resultValue.set(null);
+                  readyFuture.set(null);
+                  return;
+                }
+
+                // On error, simply invalidate the key so that it'll be fetched next time.
+                invalidater.invalidate(key);
+                readyFuture.setException(t);
+              }
+            };
+
+            // Fetch node data
+            Futures.addCallback(zkClient.getData(path, new Watcher() {
+              @Override
+              public void process(WatchedEvent event) {
+                if (!isRunning()) {
+                  return;
+                }
+                if (event.getType() == Event.EventType.NodeDataChanged) {
+                  // If node data changed, fetch it again.
+                  Futures.addCallback(zkClient.getData(path, this), dataCallback, executorService);
+                } else if (event.getType() == Event.EventType.NodeDeleted) {
+                  // If node removed, invalidate the cached value.
+                  brokerInfos.invalidate(key);
+                }
+              }
+            }), dataCallback, executorService);
+          }
+        }, readyFuture, FAILURE_RETRY_SECONDS, TimeUnit.SECONDS);
+
+        readyFuture.get();
+        return createSupplier(resultValue);
+      }
+    };
+  }
+
+  /**
+   * Gson decode the NodeData into object.
+   * @param nodeData The data to decode
+   * @param type Object class to decode into.
+   * @param <T> Type of the object.
+   * @return The decoded object or {@code null} if node data is null.
+   */
+  private <T> T decodeNodeData(NodeData nodeData, Class<T> type) {
+    byte[] data = nodeData == null ? null : nodeData.getData();
+    if (data == null) {
+      return null;
+    }
+    return GSON.fromJson(new String(data, Charsets.UTF_8), type);
+  }
+
+  /**
+   * Checks exists of a given ZK path and execute the action when it exists.
+   */
+  private void actOnExists(final String path, final Runnable action,
+                           final SettableFuture<?> readyFuture, final long retryTime, final TimeUnit retryUnit) {
+    Futures.addCallback(zkClient.exists(path, new Watcher() {
+      @Override
+      public void process(WatchedEvent event) {
+        if (!isRunning()) {
+          return;
+        }
+        if (event.getType() == Event.EventType.NodeCreated) {
+          action.run();
+        }
+      }
+    }), new FutureCallback<Stat>() {
+      @Override
+      public void onSuccess(Stat result) {
+        if (result != null) {
+          action.run();
+        } else {
+          // If the node doesn't exists, treat it as ready. When the node becomes available later, data will be
+          // fetched by the watcher.
+          readyFuture.set(null);
+        }
+      }
+
+      @Override
+      public void onFailure(Throwable t) {
+        // Retry the operation based on the retry time.
+        Thread retryThread = new Thread("zk-broker-service-retry") {
+          @Override
+          public void run() {
+            try {
+              retryUnit.sleep(retryTime);
+              actOnExists(path, action, readyFuture, retryTime, retryUnit);
+            } catch (InterruptedException e) {
+              LOG.warn("ZK retry thread interrupted. Action not retried.");
+            }
+          }
+        };
+        retryThread.setDaemon(true);
+        retryThread.start();
+      }
+    }, executorService);
+  }
+
+  /**
+   * Creates a supplier that always return latest copy from an {@link java.util.concurrent.atomic.AtomicReference}.
+   */
+  private <T> Supplier<T> createSupplier(final AtomicReference<T> ref) {
+    return new Supplier<T>() {
+      @Override
+      public T get() {
+        return ref.get();
+      }
+    };
+  }
+
+
+  /**
+   * Interface for invalidating an entry in a cache.
+   * @param <T> Key type.
+   */
+  private interface CacheInvalidater<T> {
+    void invalidate(T key);
+  }
+
+  /**
+   * Represents a path in zookeeper for cache key.
+   */
+  private interface KeyPath {
+    String getPath();
+  }
+
+  private static final class BrokerId implements KeyPath {
+    private final int id;
+
+    private BrokerId(int id) {
+      this.id = id;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      return this == o || !(o == null || getClass() != o.getClass()) && id == ((BrokerId) o).id;
+    }
+
+    @Override
+    public int hashCode() {
+      return Ints.hashCode(id);
+    }
+
+    @Override
+    public String getPath() {
+      return BROKER_IDS_PATH + "/" + id;
+    }
+  }
+
+  /**
+   * Represents a topic + partition combination. Used for loading cache key.
+   */
+  private static final class KeyPathTopicPartition extends TopicPartition implements KeyPath {
+
+    private KeyPathTopicPartition(String topic, int partition) {
+      super(topic, partition);
+    }
+
+    @Override
+    public String getPath() {
+      return String.format("%s/%s/partitions/%d/state", BROKER_TOPICS_PATH, getTopic(), getPartition());
+    }
+  }
+
+  /**
+   * Class for holding information about a partition. Only used by gson to decode partition state node in zookeeper.
+   */
+  private static final class PartitionInfo {
+    private int[] isr;
+    private int leader;
+
+    private int[] getIsr() {
+      return isr;
+    }
+
+    private int getLeader() {
+      return leader;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ZKKafkaClientService.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ZKKafkaClientService.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ZKKafkaClientService.java
new file mode 100644
index 0000000..56f812b
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ZKKafkaClientService.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.twill.internal.kafka.client;
+
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.AbstractIdleService;
+import org.apache.twill.common.Cancellable;
+import org.apache.twill.common.Threads;
+import org.apache.twill.kafka.client.BrokerService;
+import org.apache.twill.kafka.client.Compression;
+import org.apache.twill.kafka.client.KafkaClientService;
+import org.apache.twill.kafka.client.KafkaConsumer;
+import org.apache.twill.kafka.client.KafkaPublisher;
+import org.apache.twill.zookeeper.ZKClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.ref.Reference;
+import java.lang.ref.ReferenceQueue;
+import java.lang.ref.WeakReference;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A KafkaClientService that uses ZooKeeper for broker discovery.
+ */
+public class ZKKafkaClientService extends AbstractIdleService implements KafkaClientService, Runnable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ZKKafkaClientService.class);
+  private static final long PUBLISHER_CLEANUP_SECONDS = 1;
+
+  private final BrokerService brokerService;
+
+  // Maintains a weak reference key map for calling publisher.shutdown when garbage collected.
+  private final Map<WeakReference<KafkaPublisher>, Cancellable> publishers;
+  private final ReferenceQueue<KafkaPublisher> referenceQueue;
+
+  private final SimpleKafkaConsumer consumer;
+
+  // For running cleanup job
+  private ScheduledExecutorService scheduler;
+
+  public ZKKafkaClientService(ZKClient zkClient) {
+    this.brokerService = new ZKBrokerService(zkClient);
+    this.publishers = Collections.synchronizedMap(new IdentityHashMap<WeakReference<KafkaPublisher>, Cancellable>());
+    this.referenceQueue = new ReferenceQueue<KafkaPublisher>();
+    this.consumer = new SimpleKafkaConsumer(brokerService);
+  }
+
+  @Override
+  public KafkaPublisher getPublisher(KafkaPublisher.Ack ack, Compression compression) {
+    Preconditions.checkState(isRunning(), "Service is not running.");
+    String brokerList = brokerService.getBrokerList();
+    Preconditions.checkState(!brokerList.isEmpty(), "No broker available. Try again later.");
+
+    // Wrap the publisher with a weak reference and save the cancellable for closing the publisher.
+    SimpleKafkaPublisher publisher = new SimpleKafkaPublisher(brokerList, ack, compression);
+    publishers.put(new WeakReference<KafkaPublisher>(publisher, referenceQueue), publisher.start());
+    return publisher;
+  }
+
+  @Override
+  public KafkaConsumer getConsumer() {
+    Preconditions.checkState(isRunning(), "Service is not running.");
+    return consumer;
+  }
+
+  @Override
+  public void run() {
+    // For calling publisher.producer.close() on garbage collected
+    Reference<? extends KafkaPublisher> ref = referenceQueue.poll();
+    while (ref != null && isRunning()) {
+      publishers.remove(ref).cancel();
+      ref = referenceQueue.poll();
+    }
+  }
+
+  @Override
+  protected void startUp() throws Exception {
+    scheduler = Executors.newSingleThreadScheduledExecutor(Threads.createDaemonThreadFactory("kafka-client-cleanup"));
+    scheduler.scheduleAtFixedRate(this, PUBLISHER_CLEANUP_SECONDS, PUBLISHER_CLEANUP_SECONDS, TimeUnit.SECONDS);
+
+    // Start broker service to get auto-updated brokers information.
+    brokerService.startAndWait();
+  }
+
+  @Override
+  protected void shutDown() throws Exception {
+    LOG.info("Stopping KafkaClientService");
+    scheduler.shutdownNow();
+    for (Cancellable cancellable : publishers.values()) {
+      cancellable.cancel();
+    }
+    consumer.stop();
+
+    brokerService.stopAndWait();
+    LOG.info("KafkaClientService stopped");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/package-info.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/package-info.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/package-info.java
index f3f615c..8676765 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/package-info.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/package-info.java
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * This package provides pure java kafka client implementation.
+ * This package provides Kafka client implementation.
  */
 package org.apache.twill.internal.kafka.client;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java b/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
index 12818ef..9a22bea 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
@@ -17,16 +17,6 @@
  */
 package org.apache.twill.internal.logging;
 
-import org.apache.twill.common.Services;
-import org.apache.twill.common.Threads;
-import org.apache.twill.internal.kafka.client.Compression;
-import org.apache.twill.internal.kafka.client.SimpleKafkaClient;
-import org.apache.twill.kafka.client.KafkaClient;
-import org.apache.twill.kafka.client.PreparePublish;
-import org.apache.twill.zookeeper.RetryStrategies;
-import org.apache.twill.zookeeper.ZKClientService;
-import org.apache.twill.zookeeper.ZKClientServices;
-import org.apache.twill.zookeeper.ZKClients;
 import ch.qos.logback.classic.pattern.ClassOfCallerConverter;
 import ch.qos.logback.classic.pattern.FileOfCallerConverter;
 import ch.qos.logback.classic.pattern.LineOfCallerConverter;
@@ -36,7 +26,6 @@ import ch.qos.logback.classic.spi.IThrowableProxy;
 import ch.qos.logback.classic.spi.StackTraceElementProxy;
 import ch.qos.logback.core.AppenderBase;
 import com.google.common.base.Charsets;
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import com.google.common.collect.Iterables;
@@ -44,6 +33,16 @@ import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.gson.stream.JsonWriter;
+import org.apache.twill.common.Services;
+import org.apache.twill.common.Threads;
+import org.apache.twill.internal.kafka.client.ZKKafkaClientService;
+import org.apache.twill.kafka.client.Compression;
+import org.apache.twill.kafka.client.KafkaClientService;
+import org.apache.twill.kafka.client.KafkaPublisher;
+import org.apache.twill.zookeeper.RetryStrategies;
+import org.apache.twill.zookeeper.ZKClientService;
+import org.apache.twill.zookeeper.ZKClientServices;
+import org.apache.twill.zookeeper.ZKClients;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -65,7 +64,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
   private static final Logger LOG = LoggerFactory.getLogger(KafkaAppender.class);
 
   private final LogEventConverter eventConverter;
-  private final AtomicReference<PreparePublish> publisher;
+  private final AtomicReference<KafkaPublisher.Preparer> publisher;
   private final Runnable flushTask;
   /**
    * Rough count of how many entries are being buffered. It's just approximate, not exact.
@@ -73,7 +72,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
   private final AtomicInteger bufferedSize;
 
   private ZKClientService zkClientService;
-  private KafkaClient kafkaClient;
+  private KafkaClientService kafkaClient;
   private String zkConnectStr;
   private String hostname;
   private String topic;
@@ -84,7 +83,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
 
   public KafkaAppender() {
     eventConverter = new LogEventConverter();
-    publisher = new AtomicReference<PreparePublish>();
+    publisher = new AtomicReference<KafkaPublisher.Preparer>();
     flushTask = createFlushTask();
     bufferedSize = new AtomicInteger();
     buffer = new ConcurrentLinkedQueue<String>();
@@ -141,12 +140,14 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
         ZKClients.retryOnFailure(ZKClientService.Builder.of(zkConnectStr).build(),
                                  RetryStrategies.fixDelay(1, TimeUnit.SECONDS))));
 
-    kafkaClient = new SimpleKafkaClient(zkClientService);
+    kafkaClient = new ZKKafkaClientService(zkClientService);
     Futures.addCallback(Services.chainStart(zkClientService, kafkaClient), new FutureCallback<Object>() {
       @Override
       public void onSuccess(Object result) {
         LOG.info("Kafka client started: " + zkConnectStr);
-        publisher.set(kafkaClient.preparePublish(topic, Compression.SNAPPY));
+        KafkaPublisher.Preparer preparer = kafkaClient.getPublisher(KafkaPublisher.Ack.LEADER_RECEIVED,
+                                                                    Compression.SNAPPY).prepare(topic);
+        publisher.set(preparer);
         scheduler.scheduleWithFixedDelay(flushTask, 0, flushPeriod, TimeUnit.MILLISECONDS);
       }
 
@@ -186,7 +187,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
 
   private ListenableFuture<Integer> publishLogs() {
     // If the publisher is not available, simply returns a completed future.
-    PreparePublish publisher = KafkaAppender.this.publisher.get();
+    KafkaPublisher.Preparer publisher = KafkaAppender.this.publisher.get();
     if (publisher == null) {
       return Futures.immediateFuture(0);
     }
@@ -202,13 +203,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
     }
 
     bufferedSize.set(0);
-    final int finalCount = count;
-    return Futures.transform(publisher.publish(), new Function<Object, Integer>() {
-      @Override
-      public Integer apply(Object input) {
-        return finalCount;
-      }
-    });
+    return publisher.send();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaTwillRunnable.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaTwillRunnable.java b/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaTwillRunnable.java
deleted file mode 100644
index c1695de..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaTwillRunnable.java
+++ /dev/null
@@ -1,122 +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.twill.internal.logging;
-
-import org.apache.twill.api.Command;
-import org.apache.twill.api.TwillContext;
-import org.apache.twill.api.TwillRunnable;
-import org.apache.twill.api.TwillRunnableSpecification;
-import org.apache.twill.internal.EnvKeys;
-import org.apache.twill.internal.kafka.EmbeddedKafkaServer;
-import org.apache.twill.internal.utils.Networks;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableMap;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.CountDownLatch;
-
-/**
- * A {@link org.apache.twill.api.TwillRunnable} for managing Kafka server.
- */
-public final class KafkaTwillRunnable implements TwillRunnable {
-
-  private static final Logger LOG = LoggerFactory.getLogger(KafkaTwillRunnable.class);
-
-  private final String kafkaDir;
-  private EmbeddedKafkaServer server;
-  private CountDownLatch stopLatch;
-
-  public KafkaTwillRunnable(String kafkaDir) {
-    this.kafkaDir = kafkaDir;
-  }
-
-  @Override
-  public TwillRunnableSpecification configure() {
-    return TwillRunnableSpecification.Builder.with()
-      .setName("kafka")
-      .withConfigs(ImmutableMap.of("kafkaDir", kafkaDir))
-      .build();
-  }
-
-  @Override
-  public void initialize(TwillContext context) {
-    Map<String, String> args = context.getSpecification().getConfigs();
-    String zkConnectStr = System.getenv(EnvKeys.TWILL_LOG_KAFKA_ZK);
-    stopLatch = new CountDownLatch(1);
-
-    try {
-      server = new EmbeddedKafkaServer(new File(args.get("kafkaDir")), generateKafkaConfig(zkConnectStr));
-      server.startAndWait();
-    } catch (Exception e) {
-      throw Throwables.propagate(e);
-    }
-  }
-
-  @Override
-  public void handleCommand(Command command) throws Exception {
-  }
-
-  @Override
-  public void stop() {
-    stopLatch.countDown();
-  }
-
-  @Override
-  public void destroy() {
-    server.stopAndWait();
-  }
-
-  @Override
-  public void run() {
-    try {
-      stopLatch.await();
-    } catch (InterruptedException e) {
-      LOG.info("Running thread interrupted, shutting down kafka server.", e);
-    }
-  }
-
-  private Properties generateKafkaConfig(String zkConnectStr) {
-    int port = Networks.getRandomPort();
-    Preconditions.checkState(port > 0, "Failed to get random port.");
-
-    Properties prop = new Properties();
-    prop.setProperty("log.dir", new File("kafka-logs").getAbsolutePath());
-    prop.setProperty("zk.connect", zkConnectStr);
-    prop.setProperty("num.threads", "8");
-    prop.setProperty("port", Integer.toString(port));
-    prop.setProperty("log.flush.interval", "10000");
-    prop.setProperty("max.socket.request.bytes", "104857600");
-    prop.setProperty("log.cleanup.interval.mins", "1");
-    prop.setProperty("log.default.flush.scheduler.interval.ms", "1000");
-    prop.setProperty("zk.connectiontimeout.ms", "1000000");
-    prop.setProperty("socket.receive.buffer", "1048576");
-    prop.setProperty("enable.zookeeper", "true");
-    prop.setProperty("log.retention.hours", "168");
-    prop.setProperty("brokerid", "0");
-    prop.setProperty("socket.send.buffer", "1048576");
-    prop.setProperty("num.partitions", "1");
-    prop.setProperty("log.file.size", "536870912");
-    prop.setProperty("log.default.flush.interval.ms", "1000");
-    return prop;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/BrokerInfo.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/BrokerInfo.java b/twill-core/src/main/java/org/apache/twill/kafka/client/BrokerInfo.java
new file mode 100644
index 0000000..e659ab7
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/kafka/client/BrokerInfo.java
@@ -0,0 +1,65 @@
+/*
+ * 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.twill.kafka.client;
+
+import com.google.common.base.Objects;
+
+/**
+ * Represents broker information. This class should be serializable with Gson.
+ */
+public final class BrokerInfo {
+
+  private String host;
+  private int port;
+
+  public String getHost() {
+    return host;
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    BrokerInfo that = (BrokerInfo) o;
+    return host.equals(that.getHost()) && port == that.getPort();
+  }
+
+  @Override
+  public int hashCode() {
+    int result = host.hashCode();
+    result = 31 * result + port;
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return Objects.toStringHelper(BrokerInfo.class)
+                  .add("host", host)
+                  .add("port", port)
+                  .toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/BrokerService.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/BrokerService.java b/twill-core/src/main/java/org/apache/twill/kafka/client/BrokerService.java
new file mode 100644
index 0000000..97270bd
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/kafka/client/BrokerService.java
@@ -0,0 +1,48 @@
+/*
+ * 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.twill.kafka.client;
+
+import com.google.common.util.concurrent.Service;
+
+/**
+ * Service for providing information of kafka brokers.
+ */
+public interface BrokerService extends Service {
+
+  /**
+   * Returns the broker information of the current leader of the given topic and partition.
+   * @param topic Topic for looking up for leader.
+   * @param partition Partition for looking up for leader.
+   * @return A {@link BrokerInfo} containing information about the current leader, or {@code null} if
+   *         current leader is unknown.
+   */
+  BrokerInfo getLeader(String topic, int partition);
+
+  /**
+   * Returns a live iterable that gives information for all the known brokers.
+   * @return An {@link Iterable} of {@link BrokerInfo} that when {@link Iterable#iterator()} is called, it returns
+   *         an iterator that gives the latest list of {@link BrokerInfo}.
+   */
+  Iterable<BrokerInfo> getBrokers();
+
+  /**
+   * Returns a comma separate string of all current brokers.
+   * @return A string in the format {@code host1:port1,host2:port2} or empty string if no broker has been discovered.
+   */
+  String getBrokerList();
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/Compression.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/Compression.java b/twill-core/src/main/java/org/apache/twill/kafka/client/Compression.java
new file mode 100644
index 0000000..8597d3b
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/kafka/client/Compression.java
@@ -0,0 +1,38 @@
+/*
+ * 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.twill.kafka.client;
+
+/**
+ * An Enum representing different compression codec options for publishing messages.
+ */
+public enum Compression {
+
+  NONE,
+  GZIP,
+  SNAPPY;
+
+  private final String codec;
+
+  private Compression() {
+    this.codec = name().toLowerCase();
+  }
+
+  public String getCodec() {
+    return codec;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/FetchException.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/FetchException.java b/twill-core/src/main/java/org/apache/twill/kafka/client/FetchException.java
deleted file mode 100644
index acccf04..0000000
--- a/twill-core/src/main/java/org/apache/twill/kafka/client/FetchException.java
+++ /dev/null
@@ -1,77 +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.twill.kafka.client;
-
-/**
- *
- */
-public final class FetchException extends RuntimeException {
-
-  private final ErrorCode errorCode;
-
-  public FetchException(String message, ErrorCode errorCode) {
-    super(message);
-    this.errorCode = errorCode;
-  }
-
-  public ErrorCode getErrorCode() {
-    return errorCode;
-  }
-
-  @Override
-  public String toString() {
-    return String.format("%s. Error code: %s", super.toString(), errorCode);
-  }
-
-  public enum ErrorCode {
-    UNKNOWN(-1),
-    OK(0),
-    OFFSET_OUT_OF_RANGE(1),
-    INVALID_MESSAGE(2),
-    WRONG_PARTITION(3),
-    INVALID_FETCH_SIZE(4);
-
-    private final int code;
-
-    ErrorCode(int code) {
-      this.code = code;
-    }
-
-    public int getCode() {
-      return code;
-    }
-
-    public static ErrorCode fromCode(int code) {
-      switch (code) {
-        case -1:
-          return UNKNOWN;
-        case 0:
-          return OK;
-        case 1:
-          return OFFSET_OUT_OF_RANGE;
-        case 2:
-          return INVALID_MESSAGE;
-        case 3:
-          return WRONG_PARTITION;
-        case 4:
-          return INVALID_FETCH_SIZE;
-      }
-      throw new IllegalArgumentException("Unknown error code");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/FetchedMessage.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/FetchedMessage.java b/twill-core/src/main/java/org/apache/twill/kafka/client/FetchedMessage.java
index 65e140f..5739ac6 100644
--- a/twill-core/src/main/java/org/apache/twill/kafka/client/FetchedMessage.java
+++ b/twill-core/src/main/java/org/apache/twill/kafka/client/FetchedMessage.java
@@ -20,17 +20,16 @@ package org.apache.twill.kafka.client;
 import java.nio.ByteBuffer;
 
 /**
- * Represents a message fetched from kafka broker.
+ *
  */
 public interface FetchedMessage {
 
-  /**
-   * Returns the message offset.
-   */
-  long getOffset();
+  TopicPartition getTopicPartition();
+
+  ByteBuffer getPayload();
 
   /**
-   * Returns the message payload.
+   * Returns the offset for the next message to be read.
    */
-  ByteBuffer getBuffer();
+  long getNextOffset();
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaClient.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaClient.java b/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaClient.java
index 496195b..f98d879 100644
--- a/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaClient.java
+++ b/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaClient.java
@@ -17,34 +17,22 @@
  */
 package org.apache.twill.kafka.client;
 
-import org.apache.twill.internal.kafka.client.Compression;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.Service;
-
-import java.util.Iterator;
-
 /**
- * This interface provides methods for interacting with kafka broker. It also
- * extends from {@link Service} for lifecycle management. The {@link #start()} method
- * must be called prior to other methods in this class. When instance of this class
- * is not needed, call {@link #stop()}} to release any resources that it holds.
+ * Represents a kafka client that can publish/subscribe to a Kafka server cluster.
  */
-public interface KafkaClient extends Service {
-
-  PreparePublish preparePublish(String topic, Compression compression);
+public interface KafkaClient {
 
-  Iterator<FetchedMessage> consume(String topic, int partition, long offset, int maxSize);
+  /**
+   * Creates a {@link KafkaPublisher} that is ready for publish.
+   * @param ack Type of ack that the publisher would use for all it's publish.
+   * @param compression The compression type for messages published through the returned publisher.
+   * @return A {@link KafkaPublisher}.
+   */
+  KafkaPublisher getPublisher(KafkaPublisher.Ack ack, Compression compression);
 
   /**
-   * Fetches offset from the given topic and partition.
-   * @param topic Topic to fetch from.
-   * @param partition Partition to fetch from.
-   * @param time The first offset of every segment file for a given partition with a modified time less than time.
-   *             {@code -1} for latest offset, {@code -2} for earliest offset.
-   * @param maxOffsets Maximum number of offsets to fetch.
-   * @return A Future that carry the result as an array of offsets in descending order.
-   *         The size of the result array would not be larger than maxOffsets. If there is any error during the fetch,
-   *         the exception will be carried in the exception.
+   * Creates a {@link KafkaConsumer} for consuming messages.
+   * @return A {@link KafkaConsumer}.
    */
-  ListenableFuture<long[]> getOffset(String topic, int partition, long time, int maxOffsets);
+  KafkaConsumer getConsumer();
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaClientService.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaClientService.java b/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaClientService.java
new file mode 100644
index 0000000..d9224e0
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/kafka/client/KafkaClientService.java
@@ -0,0 +1,26 @@
+/*
+ * 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.twill.kafka.client;
+
+import com.google.common.util.concurrent.Service;
+
+/**
+ * A {@link KafkaClient} that extends {@link Service} to provide lifecycle management.
+ */
+public interface KafkaClientService extends KafkaClient, Service {
+}


[23/50] [abbrv] git commit: Enable RAT check for the apache-release profile. Also make RAT checks failure fails the build.

Posted by ch...@apache.org.
Enable RAT check for the apache-release profile. Also make RAT checks failure fails the build.

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

Branch: refs/heads/site
Commit: 64a01403bd0dcf2811494a87a648acc3f3e0cfbd
Parents: 5e89ec2
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Feb 10 14:02:00 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Feb 10 14:02:00 2014 -0800

----------------------------------------------------------------------
 pom.xml | 41 ++++++++++++++++++++++++++---------------
 1 file changed, 26 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/64a01403/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1d0a816..b1c4bfe 100644
--- a/pom.xml
+++ b/pom.xml
@@ -244,6 +244,20 @@
                         </execution>
                     </executions>
                 </plugin>
+                <plugin>
+                    <groupId>org.apache.rat</groupId>
+                    <artifactId>apache-rat-plugin</artifactId>
+                    <version>0.10</version>
+                    <configuration>
+                        <excludes>
+                            <exclude>.git/**/*</exclude>
+                            <exclude>**/*.iml</exclude>
+                            <exclude>target/**/*</exclude>
+                            <exclude>**/README</exclude>
+                            <exclude>src/test/resources/header.txt</exclude>
+                        </excludes>
+                    </configuration>
+                </plugin>
             </plugins>
         </pluginManagement>
         <plugins>
@@ -314,21 +328,6 @@
                     <deployAtEnd>true</deployAtEnd>
                 </configuration>
             </plugin>
-            <plugin>
-                <groupId>org.apache.rat</groupId>
-                <artifactId>apache-rat-plugin</artifactId>
-                <version>0.10</version>
-                <configuration>
-                    <excludes>
-                        <exclude>.git/**/*</exclude>
-                        <exclude>**/*.iml</exclude>
-                        <exclude>target/**/*</exclude>
-                        <exclude>**/README</exclude>
-                        <exclude>src/test/resources/header.txt</exclude>
-                    </excludes>
-                    <ignoreErrors>true</ignoreErrors>
-                </configuration>
-            </plugin>
         </plugins>
     </build>
 
@@ -348,6 +347,18 @@
                             </formats>
                         </configuration>
                     </plugin>
+                    <plugin>
+                        <groupId>org.apache.rat</groupId>
+                        <artifactId>apache-rat-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <phase>prepare-package</phase>
+                                <goals>
+                                    <goal>check</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
                 </plugins>
             </build>
         </profile>


[47/50] [abbrv] git commit: Added "Apache" to maven submodules names.

Posted by ch...@apache.org.
Added "Apache" to maven submodules names.


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

Branch: refs/heads/site
Commit: d6504ebeb84a596a10655d7176ebf4d4de5fa319
Parents: a028289
Author: Terence Yim <ch...@apache.org>
Authored: Tue Apr 15 00:06:50 2014 -0700
Committer: Terence Yim <ch...@apache.org>
Committed: Tue Apr 15 00:09:38 2014 -0700

----------------------------------------------------------------------
 twill-api/pom.xml            | 2 +-
 twill-common/pom.xml         | 2 +-
 twill-core/pom.xml           | 2 +-
 twill-discovery-api/pom.xml  | 2 +-
 twill-discovery-core/pom.xml | 2 +-
 twill-examples/echo/pom.xml  | 6 +++---
 twill-examples/pom.xml       | 6 +++---
 twill-examples/yarn/pom.xml  | 6 +++---
 twill-ext/pom.xml            | 4 ++--
 twill-yarn/pom.xml           | 2 +-
 twill-zookeeper/pom.xml      | 2 +-
 11 files changed, 18 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d6504ebe/twill-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-api/pom.xml b/twill-api/pom.xml
index 250289c..570d864 100644
--- a/twill-api/pom.xml
+++ b/twill-api/pom.xml
@@ -29,7 +29,7 @@
 
     <artifactId>twill-api</artifactId>
     <packaging>jar</packaging>
-    <name>Twill API</name>
+    <name>Apache Twill API</name>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d6504ebe/twill-common/pom.xml
----------------------------------------------------------------------
diff --git a/twill-common/pom.xml b/twill-common/pom.xml
index e9e641f..e33e002 100644
--- a/twill-common/pom.xml
+++ b/twill-common/pom.xml
@@ -27,7 +27,7 @@
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>twill-common</artifactId>
-    <name>Twill common library</name>
+    <name>Apache Twill common library</name>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d6504ebe/twill-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-core/pom.xml b/twill-core/pom.xml
index 7b7a085..2ff3aa9 100644
--- a/twill-core/pom.xml
+++ b/twill-core/pom.xml
@@ -27,7 +27,7 @@
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>twill-core</artifactId>
-    <name>Twill core library</name>
+    <name>Apache Twill core library</name>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d6504ebe/twill-discovery-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-api/pom.xml b/twill-discovery-api/pom.xml
index 0ce6c38..8d9552a 100644
--- a/twill-discovery-api/pom.xml
+++ b/twill-discovery-api/pom.xml
@@ -27,7 +27,7 @@
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>twill-discovery-api</artifactId>
-    <name>Twill discovery service API</name>
+    <name>Apache Twill discovery service API</name>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d6504ebe/twill-discovery-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-core/pom.xml b/twill-discovery-core/pom.xml
index 2bdad6b..4b5d101 100644
--- a/twill-discovery-core/pom.xml
+++ b/twill-discovery-core/pom.xml
@@ -27,7 +27,7 @@
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>twill-discovery-core</artifactId>
-    <name>Twill discovery service implementations</name>
+    <name>Apache Twill discovery service implementations</name>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d6504ebe/twill-examples/echo/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/echo/pom.xml b/twill-examples/echo/pom.xml
index 8e665ab..bd52755 100644
--- a/twill-examples/echo/pom.xml
+++ b/twill-examples/echo/pom.xml
@@ -25,10 +25,10 @@ limitations under the License.
     <parent>
         <artifactId>twill-examples</artifactId>
         <groupId>org.apache.twill.example</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
 
-    <name>Twill examples: Echo</name>
+    <name>Apache Twill examples: Echo</name>
     <description>
         Simple echo application that echos the command line arguments passed to it.
         Used to demonstrate usage of BundledJarApplication to run applications that have
@@ -72,4 +72,4 @@ limitations under the License.
             </plugin>
         </plugins>
     </build>
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d6504ebe/twill-examples/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/pom.xml b/twill-examples/pom.xml
index 79a9581..b35816e 100644
--- a/twill-examples/pom.xml
+++ b/twill-examples/pom.xml
@@ -24,7 +24,7 @@ limitations under the License.
     <groupId>org.apache.twill.example</groupId>
     <artifactId>twill-examples</artifactId>
     <version>0.3.0-incubating-SNAPSHOT</version>
-    <name>Twill examples</name>
+    <name>Apache Twill examples</name>
     <packaging>pom</packaging>
 
     <modules>
@@ -33,8 +33,8 @@ limitations under the License.
     </modules>
 
     <properties>
-        <twill.version>0.2.0-incubating-SNAPSHOT</twill.version>
+        <twill.version>0.3.0-incubating-SNAPSHOT</twill.version>
         <twill.groupId>org.apache.twill</twill.groupId>
     </properties>
 
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d6504ebe/twill-examples/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/yarn/pom.xml b/twill-examples/yarn/pom.xml
index 3deea44..00e5775 100644
--- a/twill-examples/yarn/pom.xml
+++ b/twill-examples/yarn/pom.xml
@@ -24,10 +24,10 @@ limitations under the License.
     <parent>
         <artifactId>twill-examples</artifactId>
         <groupId>org.apache.twill.example</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
 
-    <name>Twill examples: YARN</name>
+    <name>Apache Twill examples: YARN</name>
     <description>Examples demonstrating usage of twill-yarn</description>
     <artifactId>twill-examples-yarn</artifactId>
 
@@ -79,4 +79,4 @@ limitations under the License.
             </plugin>
         </plugins>
     </build>
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d6504ebe/twill-ext/pom.xml
----------------------------------------------------------------------
diff --git a/twill-ext/pom.xml b/twill-ext/pom.xml
index 3d835bf..2389de2 100644
--- a/twill-ext/pom.xml
+++ b/twill-ext/pom.xml
@@ -27,7 +27,7 @@ limitations under the License.
 
     <modelVersion>4.0.0</modelVersion>
     <artifactId>twill-ext</artifactId>
-    <name>Twill extensions</name>
+    <name>Apache Twill extensions</name>
 
     <dependencies>
         <dependency>
@@ -54,4 +54,4 @@ limitations under the License.
         </dependency>
     </dependencies>
 
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d6504ebe/twill-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-yarn/pom.xml b/twill-yarn/pom.xml
index 014489c..1d70645 100644
--- a/twill-yarn/pom.xml
+++ b/twill-yarn/pom.xml
@@ -27,7 +27,7 @@
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>twill-yarn</artifactId>
-    <name>Twill Apache Hadoop YARN library</name>
+    <name>Apache Twill Apache Hadoop YARN library</name>
 
     <properties>
         <output.dir>target/classes</output.dir>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d6504ebe/twill-zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/twill-zookeeper/pom.xml b/twill-zookeeper/pom.xml
index 8b1f4fe..1b37cc5 100644
--- a/twill-zookeeper/pom.xml
+++ b/twill-zookeeper/pom.xml
@@ -27,7 +27,7 @@
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>twill-zookeeper</artifactId>
-    <name>Twill ZooKeeper client library</name>
+    <name>Apache Twill ZooKeeper client library</name>
 
     <dependencies>
         <dependency>


[02/50] [abbrv] git commit: Added Disclaimer to source and to web-site.

Posted by ch...@apache.org.
Added Disclaimer to source and to web-site.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 9fe7b7041d1412920d0b85cd328a714b6e2ca816
Parents: b0dd8e2
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Jan 20 11:00:30 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Jan 20 14:55:11 2014 -0800

----------------------------------------------------------------------
 DISCLAIMER                 |  1 +
 src/site/markdown/index.md | 10 +++++++++-
 src/site/site.xml          |  8 +++++---
 3 files changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9fe7b704/DISCLAIMER
----------------------------------------------------------------------
diff --git a/DISCLAIMER b/DISCLAIMER
new file mode 100644
index 0000000..ead0dc9
--- /dev/null
+++ b/DISCLAIMER
@@ -0,0 +1 @@
+Apache Twill is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9fe7b704/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/index.md b/src/site/markdown/index.md
index 06eb967..56ce3bf 100644
--- a/src/site/markdown/index.md
+++ b/src/site/markdown/index.md
@@ -24,4 +24,12 @@
 
 Twill is an abstraction over Apache Hadoop® YARN that reduces the complexity of developing distributed applications,
 allowing developers to focus more on their business logic. Twill allows you to use YARN’s distributed capabilities
-with a programming model that is similar to running threads.
\ No newline at end of file
+with a programming model that is similar to running threads.
+
+### Disclaimer
+
+Apache Twill is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by Incubator.
+Incubation is required of all newly accepted projects until a further review indicates that the infrastructure,
+communications, and decision making process have stabilized in a manner consistent with other successful ASF projects.
+While incubation status is not necessarily a reflection of the completeness or stability of the code,
+it does indicate that the project has yet to be fully endorsed by the ASF.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9fe7b704/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index c400c01..d5e1735 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -61,9 +61,11 @@
 
         <footer>
             <div class="row span16">
-                Apache Twill, Apache, the Apache feather logo,
-                and the Apache Twill project logos are trademarks of The Apache Software Foundation.
-                All other marks mentioned may be trademarks or registered trademarks of their respective owners.
+                <div>
+                    Apache Twill, Apache, the Apache feather logo,
+                    and the Apache Twill project logos are trademarks of The Apache Software Foundation.
+                    All other marks mentioned may be trademarks or registered trademarks of their respective owners.
+                </div>
                 <a href="${project.url}/privacy-policy.html">Privacy Policy</a>
             </div>
         </footer>


[06/50] [abbrv] git commit: [TWILL-27] Upgrade to Kafka-0.8

Posted by ch...@apache.org.
[TWILL-27] Upgrade to Kafka-0.8

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: fcbe54d50d22a4a85eaea35612cd9db143deef67
Parents: 15de6ae
Author: Terence Yim <te...@continuuity.com>
Authored: Wed Jan 22 17:07:52 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Jan 27 13:40:28 2014 -0800

----------------------------------------------------------------------
 pom.xml                                         |  19 +
 twill-core/pom.xml                              |   4 +
 .../twill/internal/AbstractTwillController.java | 124 ++---
 .../org/apache/twill/internal/Constants.java    |   1 -
 .../internal/kafka/EmbeddedKafkaServer.java     |  64 +--
 .../AbstractCompressedMessageSetEncoder.java    |  78 ---
 .../kafka/client/AbstractMessageSetEncoder.java |  79 ---
 .../kafka/client/BasicFetchedMessage.java       |  34 +-
 .../twill/internal/kafka/client/Bufferer.java   |  61 ---
 .../kafka/client/ByteBufferEncoder.java         |  38 ++
 .../internal/kafka/client/Compression.java      |  49 --
 .../internal/kafka/client/ConnectionPool.java   | 125 -----
 .../kafka/client/GZipMessageSetEncoder.java     |  37 --
 .../kafka/client/IdentityMessageSetEncoder.java |  42 --
 .../internal/kafka/client/IntegerEncoder.java   |  35 ++
 .../kafka/client/IntegerPartitioner.java        |  34 ++
 .../internal/kafka/client/KafkaBrokerCache.java | 326 -------------
 .../internal/kafka/client/KafkaRequest.java     |  91 ----
 .../kafka/client/KafkaRequestEncoder.java       |  60 ---
 .../kafka/client/KafkaRequestSender.java        |  26 -
 .../internal/kafka/client/KafkaResponse.java    |  49 --
 .../kafka/client/KafkaResponseDispatcher.java   |  63 ---
 .../kafka/client/KafkaResponseHandler.java      |  51 --
 .../internal/kafka/client/MessageFetcher.java   | 243 ----------
 .../kafka/client/MessageSetEncoder.java         |  31 --
 .../internal/kafka/client/ResponseHandler.java  |  33 --
 .../kafka/client/SimpleKafkaClient.java         | 304 ------------
 .../kafka/client/SimpleKafkaConsumer.java       | 481 +++++++++++++++++++
 .../kafka/client/SimpleKafkaPublisher.java      | 113 +++++
 .../kafka/client/SnappyMessageSetEncoder.java   |  38 --
 .../internal/kafka/client/TopicBroker.java      |  48 --
 .../internal/kafka/client/ZKBrokerService.java  | 403 ++++++++++++++++
 .../kafka/client/ZKKafkaClientService.java      | 118 +++++
 .../internal/kafka/client/package-info.java     |   2 +-
 .../twill/internal/logging/KafkaAppender.java   |  43 +-
 .../internal/logging/KafkaTwillRunnable.java    | 122 -----
 .../apache/twill/kafka/client/BrokerInfo.java   |  65 +++
 .../twill/kafka/client/BrokerService.java       |  48 ++
 .../apache/twill/kafka/client/Compression.java  |  38 ++
 .../twill/kafka/client/FetchException.java      |  77 ---
 .../twill/kafka/client/FetchedMessage.java      |  13 +-
 .../apache/twill/kafka/client/KafkaClient.java  |  36 +-
 .../twill/kafka/client/KafkaClientService.java  |  26 +
 .../twill/kafka/client/KafkaConsumer.java       |  92 ++++
 .../twill/kafka/client/KafkaPublisher.java      |  95 ++++
 .../twill/kafka/client/PreparePublish.java      |  34 --
 .../twill/kafka/client/TopicPartition.java      |  70 +++
 .../apache/twill/kafka/client/package-info.java |   2 +-
 twill-core/src/main/resources/kafka-0.7.2.tgz   | Bin 8811693 -> 0 bytes
 .../apache/twill/kafka/client/KafkaTest.java    | 139 ++----
 .../appmaster/ApplicationMasterService.java     |  93 ++--
 .../apache/twill/yarn/YarnTwillPreparer.java    |  10 -
 52 files changed, 1889 insertions(+), 2418 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0c6e9dd..3d7cebe 100644
--- a/pom.xml
+++ b/pom.xml
@@ -522,6 +522,25 @@
                 <version>4.0</version>
             </dependency>
             <dependency>
+                <groupId>org.apache.kafka</groupId>
+                <artifactId>kafka_2.10</artifactId>
+                <version>0.8.0</version>
+                <exclusions>
+                    <exclusion>
+                        <groupId>org.slf4j</groupId>
+                        <artifactId>slf4j-simple</artifactId>
+                    </exclusion>
+                    <exclusion>
+                        <groupId>com.sun.jdmk</groupId>
+                        <artifactId>jmxtools</artifactId>
+                    </exclusion>
+                    <exclusion>
+                        <groupId>com.sun.jmx</groupId>
+                        <artifactId>jmxri</artifactId>
+                    </exclusion>
+                </exclusions>
+            </dependency>
+            <dependency>
                 <groupId>org.apache.hadoop</groupId>
                 <artifactId>hadoop-yarn-api</artifactId>
                 <version>${hadoop.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-core/pom.xml b/twill-core/pom.xml
index faff711..d998b40 100644
--- a/twill-core/pom.xml
+++ b/twill-core/pom.xml
@@ -78,6 +78,10 @@
             <artifactId>logback-classic</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka_2.10</artifactId>
+        </dependency>
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java b/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
index 97e0a8f..71f0c14 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
@@ -17,33 +17,34 @@
  */
 package org.apache.twill.internal;
 
+import com.google.common.base.Charsets;
+import com.google.common.collect.Iterables;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.TwillController;
 import org.apache.twill.api.logging.LogEntry;
 import org.apache.twill.api.logging.LogHandler;
+import org.apache.twill.common.Cancellable;
 import org.apache.twill.discovery.DiscoveryServiceClient;
 import org.apache.twill.discovery.ServiceDiscovered;
 import org.apache.twill.discovery.ZKDiscoveryService;
 import org.apache.twill.internal.json.StackTraceElementCodec;
-import org.apache.twill.internal.kafka.client.SimpleKafkaClient;
+import org.apache.twill.internal.kafka.client.ZKKafkaClientService;
 import org.apache.twill.internal.logging.LogEntryDecoder;
 import org.apache.twill.internal.state.SystemMessages;
 import org.apache.twill.kafka.client.FetchedMessage;
-import org.apache.twill.kafka.client.KafkaClient;
+import org.apache.twill.kafka.client.KafkaClientService;
+import org.apache.twill.kafka.client.KafkaConsumer;
 import org.apache.twill.zookeeper.ZKClient;
 import org.apache.twill.zookeeper.ZKClients;
-import com.google.common.base.Charsets;
-import com.google.common.collect.Iterables;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.Iterator;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.TimeUnit;
 
 /**
  * A abstract base class for {@link org.apache.twill.api.TwillController} implementation that uses Zookeeper to controller a
@@ -52,47 +53,47 @@ import java.util.concurrent.TimeUnit;
 public abstract class AbstractTwillController extends AbstractZKServiceController implements TwillController {
 
   private static final Logger LOG = LoggerFactory.getLogger(AbstractTwillController.class);
-  private static final int MAX_KAFKA_FETCH_SIZE = 1048576;
-  private static final long SHUTDOWN_TIMEOUT_MS = 2000;
-  private static final long LOG_FETCH_TIMEOUT_MS = 5000;
 
   private final Queue<LogHandler> logHandlers;
-  private final KafkaClient kafkaClient;
+  private final KafkaClientService kafkaClient;
   private final DiscoveryServiceClient discoveryServiceClient;
-  private final LogPollerThread logPoller;
+  private volatile Cancellable logCancellable;
 
   public AbstractTwillController(RunId runId, ZKClient zkClient, Iterable<LogHandler> logHandlers) {
     super(runId, zkClient);
     this.logHandlers = new ConcurrentLinkedQueue<LogHandler>();
-    this.kafkaClient = new SimpleKafkaClient(ZKClients.namespace(zkClient, "/" + runId.getId() + "/kafka"));
+    this.kafkaClient = new ZKKafkaClientService(ZKClients.namespace(zkClient, "/" + runId.getId() + "/kafka"));
     this.discoveryServiceClient = new ZKDiscoveryService(zkClient);
     Iterables.addAll(this.logHandlers, logHandlers);
-    this.logPoller = new LogPollerThread(runId, kafkaClient, logHandlers);
   }
 
   @Override
-  protected void doStartUp() {
+  protected synchronized void doStartUp() {
     if (!logHandlers.isEmpty()) {
-      logPoller.start();
+      kafkaClient.startAndWait();
+      logCancellable = kafkaClient.getConsumer().prepare()
+                                  .addFromBeginning(Constants.LOG_TOPIC, 0)
+                                  .consume(new LogMessageCallback(logHandlers));
     }
   }
 
   @Override
   protected void doShutDown() {
-    logPoller.terminate();
-    try {
-      // Wait for the poller thread to stop.
-      logPoller.join(SHUTDOWN_TIMEOUT_MS);
-    } catch (InterruptedException e) {
-      LOG.warn("Joining of log poller thread interrupted.", e);
+    if (logCancellable != null) {
+      logCancellable.cancel();
     }
+    // Safe to call stop no matter when state the KafkaClientService is in.
+    kafkaClient.stopAndWait();
   }
 
   @Override
   public final synchronized void addLogHandler(LogHandler handler) {
     logHandlers.add(handler);
-    if (!logPoller.isAlive()) {
-      logPoller.start();
+    if (logHandlers.size() == 1) {
+      kafkaClient.startAndWait();
+      logCancellable = kafkaClient.getConsumer().prepare()
+        .addFromBeginning(Constants.LOG_TOPIC, 0)
+        .consume(new LogMessageCallback(logHandlers));
     }
   }
 
@@ -106,74 +107,45 @@ public abstract class AbstractTwillController extends AbstractZKServiceControlle
     return sendMessage(SystemMessages.setInstances(runnable, newCount), newCount);
   }
 
-  private static final class LogPollerThread extends Thread {
+  private static final class LogMessageCallback implements KafkaConsumer.MessageCallback {
+
+    private static final Gson GSON = new GsonBuilder().registerTypeAdapter(LogEntry.class, new LogEntryDecoder())
+      .registerTypeAdapter(StackTraceElement.class, new StackTraceElementCodec())
+      .create();
 
-    private final KafkaClient kafkaClient;
     private final Iterable<LogHandler> logHandlers;
-    private volatile boolean running = true;
 
-    LogPollerThread(RunId runId, KafkaClient kafkaClient, Iterable<LogHandler> logHandlers) {
-      super("twill-log-poller-" + runId.getId());
-      setDaemon(true);
-      this.kafkaClient = kafkaClient;
+    private LogMessageCallback(Iterable<LogHandler> logHandlers) {
       this.logHandlers = logHandlers;
     }
 
     @Override
-    public void run() {
-      LOG.info("Twill log poller thread '{}' started.", getName());
-      kafkaClient.startAndWait();
-      Gson gson = new GsonBuilder().registerTypeAdapter(LogEntry.class, new LogEntryDecoder())
-        .registerTypeAdapter(StackTraceElement.class, new StackTraceElementCodec())
-        .create();
-
-      while (running && !isInterrupted()) {
-        long offset;
-        try {
-          // Get the earliest offset
-          long[] offsets = kafkaClient.getOffset(Constants.LOG_TOPIC, 0, -2, 1).get(LOG_FETCH_TIMEOUT_MS,
-                                                                                    TimeUnit.MILLISECONDS);
-          // Should have one entry
-          offset = offsets[0];
-        } catch (Throwable t) {
-          // Keep retrying
-          LOG.warn("Failed to fetch offsets from Kafka. Retrying.", t);
-          continue;
-        }
-
-        // Now fetch log messages from Kafka
-        Iterator<FetchedMessage> messageIterator = kafkaClient.consume(Constants.LOG_TOPIC, 0,
-                                                                       offset, MAX_KAFKA_FETCH_SIZE);
+    public void onReceived(Iterator<FetchedMessage> messages) {
+      while (messages.hasNext()) {
+        String json = Charsets.UTF_8.decode(messages.next().getPayload()).toString();
         try {
-          while (messageIterator.hasNext()) {
-            String json = Charsets.UTF_8.decode(messageIterator.next().getBuffer()).toString();
-            try {
-              LogEntry entry = gson.fromJson(json, LogEntry.class);
-              if (entry != null) {
-                invokeHandlers(entry);
-              }
-            } catch (Exception e) {
-              LOG.error("Failed to decode log entry {}", json, e);
-            }
+          LogEntry entry = GSON.fromJson(json, LogEntry.class);
+          if (entry != null) {
+            invokeHandlers(entry);
           }
-        } catch (Throwable t) {
-          LOG.warn("Exception while fetching log message from Kafka. Retrying.", t);
-          continue;
+        } catch (Exception e) {
+          LOG.error("Failed to decode log entry {}", json, e);
         }
       }
-
-      kafkaClient.stopAndWait();
-      LOG.info("Twill log poller thread stopped.");
     }
 
-    void terminate() {
-      running = false;
-      interrupt();
+    @Override
+    public void finished() {
+      // No-op
     }
 
     private void invokeHandlers(LogEntry entry) {
       for (LogHandler handler : logHandlers) {
-        handler.onLog(entry);
+        try {
+          handler.onLog(entry);
+        } catch (Throwable t) {
+          LOG.warn("Exception while calling LogHandler {}", handler, t);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/Constants.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/Constants.java b/twill-core/src/main/java/org/apache/twill/internal/Constants.java
index 0387d3e..efe91a7 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/Constants.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/Constants.java
@@ -51,7 +51,6 @@ public final class Constants {
     public static final String TWILL_SPEC = "twillSpec.json";
     public static final String ARGUMENTS = "arguments.json";
     public static final String LOGBACK_TEMPLATE = "logback-template.xml";
-    public static final String KAFKA = "kafka.tgz";
     public static final String JVM_OPTIONS = "jvm.opts";
     public static final String CREDENTIALS = "credentials.store";
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/EmbeddedKafkaServer.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/EmbeddedKafkaServer.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/EmbeddedKafkaServer.java
index 14dfc70..d82d617 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/EmbeddedKafkaServer.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/EmbeddedKafkaServer.java
@@ -17,77 +17,33 @@
  */
 package org.apache.twill.internal.kafka;
 
-import com.google.common.base.Throwables;
-import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.AbstractIdleService;
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServerStartable;
 
 import java.io.File;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.List;
 import java.util.Properties;
 
 /**
- *
+ * A {@link com.google.common.util.concurrent.Service} implementation for running an instance of Kafka server in
+ * the same process.
  */
 public final class EmbeddedKafkaServer extends AbstractIdleService {
 
-  private static final String KAFAK_CONFIG_CLASS = "kafka.server.KafkaConfig";
-  private static final String KAFKA_SERVER_CLASS = "kafka.server.KafkaServerStartable";
-
-  private final Object server;
-
-  public EmbeddedKafkaServer(File kafkaDir, Properties properties) {
-    this(createClassLoader(kafkaDir), properties);
-  }
-
-  public EmbeddedKafkaServer(ClassLoader classLoader, Properties properties) {
-    try {
-      Class<?> configClass = classLoader.loadClass(KAFAK_CONFIG_CLASS);
-      Object config = configClass.getConstructor(Properties.class).newInstance(properties);
+  private final KafkaServerStartable server;
 
-      Class<?> serverClass = classLoader.loadClass(KAFKA_SERVER_CLASS);
-      server = serverClass.getConstructor(configClass).newInstance(config);
-    } catch (Exception e) {
-      throw Throwables.propagate(e);
-    }
+  public EmbeddedKafkaServer(Properties properties) {
+    server = new KafkaServerStartable(new KafkaConfig(properties));
   }
 
   @Override
   protected void startUp() throws Exception {
-    server.getClass().getMethod("startup").invoke(server);
+    server.startup();
   }
 
   @Override
   protected void shutDown() throws Exception {
-    server.getClass().getMethod("shutdown").invoke(server);
-    server.getClass().getMethod("awaitShutdown").invoke(server);
-  }
-
-  private static ClassLoader createClassLoader(File kafkaDir) {
-    ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
-    ClassLoader thisClassLoader = EmbeddedKafkaServer.class.getClassLoader();
-    ClassLoader parent = contextClassLoader != null
-                            ? contextClassLoader
-                            : thisClassLoader != null
-                                ? thisClassLoader : ClassLoader.getSystemClassLoader();
-
-    return new URLClassLoader(findJars(kafkaDir, Lists.<URL>newArrayList()).toArray(new URL[0]), parent);
-  }
-
-  private static List<URL> findJars(File dir, List<URL> urls) {
-    try {
-      for (File file : dir.listFiles()) {
-        if (file.isDirectory()) {
-          findJars(file, urls);
-        } else if (file.getName().endsWith(".jar")) {
-          urls.add(file.toURI().toURL());
-        }
-      }
-      return urls;
-    } catch (MalformedURLException e) {
-      throw Throwables.propagate(e);
-    }
+    server.shutdown();
+    server.awaitShutdown();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/AbstractCompressedMessageSetEncoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/AbstractCompressedMessageSetEncoder.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/AbstractCompressedMessageSetEncoder.java
deleted file mode 100644
index a9c3381..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/AbstractCompressedMessageSetEncoder.java
+++ /dev/null
@@ -1,78 +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.twill.internal.kafka.client;
-
-import com.google.common.base.Throwables;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBufferOutputStream;
-import org.jboss.netty.buffer.ChannelBuffers;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * A base implementation of {@link MessageSetEncoder} that do message compression.
- */
-abstract class AbstractCompressedMessageSetEncoder extends AbstractMessageSetEncoder {
-
-  private final Compression compression;
-  private ChannelBufferOutputStream os;
-  private OutputStream compressedOutput;
-
-
-  protected AbstractCompressedMessageSetEncoder(Compression compression) {
-    this.compression = compression;
-    try {
-      this.os = new ChannelBufferOutputStream(ChannelBuffers.dynamicBuffer());
-      this.compressedOutput = createCompressedStream(os);
-    } catch (IOException e) {
-      // Should never happen
-      throw Throwables.propagate(e);
-    }
-  }
-
-  @Override
-  public final MessageSetEncoder add(ChannelBuffer payload) {
-    try {
-      ChannelBuffer encoded = encodePayload(payload);
-      encoded.readBytes(compressedOutput, encoded.readableBytes());
-    } catch (IOException e) {
-      throw Throwables.propagate(e);
-    }
-    return this;
-
-  }
-
-  @Override
-  public final ChannelBuffer finish() {
-    try {
-      compressedOutput.close();
-      ChannelBuffer buf = prefixLength(encodePayload(os.buffer(), compression));
-      compressedOutput = createCompressedStream(os);
-      os.buffer().clear();
-
-      return buf;
-
-    } catch (IOException e) {
-      throw Throwables.propagate(e);
-    }
-
-  }
-
-  protected abstract OutputStream createCompressedStream(OutputStream os) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/AbstractMessageSetEncoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/AbstractMessageSetEncoder.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/AbstractMessageSetEncoder.java
deleted file mode 100644
index 9955d6a..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/AbstractMessageSetEncoder.java
+++ /dev/null
@@ -1,79 +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.twill.internal.kafka.client;
-
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-
-import java.util.zip.CRC32;
-
-/**
- * A base implementation of {@link MessageSetEncoder}.
- */
-abstract class AbstractMessageSetEncoder implements MessageSetEncoder {
-
-  private static final ThreadLocal<CRC32> CRC32_LOCAL = new ThreadLocal<CRC32>() {
-    @Override
-    protected CRC32 initialValue() {
-      return new CRC32();
-    }
-  };
-
-  protected final int computeCRC32(ChannelBuffer buffer) {
-    CRC32 crc32 = CRC32_LOCAL.get();
-    crc32.reset();
-
-    if (buffer.hasArray()) {
-      crc32.update(buffer.array(), buffer.arrayOffset() + buffer.readerIndex(), buffer.readableBytes());
-    } else {
-      byte[] bytes = new byte[buffer.readableBytes()];
-      buffer.getBytes(buffer.readerIndex(), bytes);
-      crc32.update(bytes);
-    }
-    return (int) crc32.getValue();
-  }
-
-  protected final ChannelBuffer encodePayload(ChannelBuffer payload) {
-    return encodePayload(payload, Compression.NONE);
-  }
-
-  protected final ChannelBuffer encodePayload(ChannelBuffer payload, Compression compression) {
-    ChannelBuffer header = ChannelBuffers.buffer(10);
-
-    int crc = computeCRC32(payload);
-
-    int magic = ((compression == Compression.NONE) ? 0 : 1);
-
-    // Message length = 1 byte magic + (optional 1 compression byte) + 4 bytes crc + payload length
-    header.writeInt(5 + magic + payload.readableBytes());
-    // Magic number = 0 for non-compressed data
-    header.writeByte(magic);
-    if (magic > 0) {
-      header.writeByte(compression.getCode());
-    }
-    header.writeInt(crc);
-
-    return ChannelBuffers.wrappedBuffer(header, payload);
-  }
-
-  protected final ChannelBuffer prefixLength(ChannelBuffer buffer) {
-    ChannelBuffer sizeBuf = ChannelBuffers.buffer(4);
-    sizeBuf.writeInt(buffer.readableBytes());
-    return ChannelBuffers.wrappedBuffer(sizeBuf, buffer);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/BasicFetchedMessage.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/BasicFetchedMessage.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/BasicFetchedMessage.java
index 286bf82..ee53ed4 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/BasicFetchedMessage.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/BasicFetchedMessage.java
@@ -18,29 +18,43 @@
 package org.apache.twill.internal.kafka.client;
 
 import org.apache.twill.kafka.client.FetchedMessage;
+import org.apache.twill.kafka.client.TopicPartition;
 
 import java.nio.ByteBuffer;
 
 /**
- *
+ * An implementation of FetchedMessage that provides setters as well.
  */
 final class BasicFetchedMessage implements FetchedMessage {
 
-  private final long offset;
-  private final ByteBuffer buffer;
+  private final TopicPartition topicPartition;
+  private ByteBuffer payload;
+  private long nextOffset;
+
+  BasicFetchedMessage(TopicPartition topicPartition) {
+    this.topicPartition = topicPartition;
+  }
+
+  void setPayload(ByteBuffer payload) {
+    this.payload = payload;
+  }
+
+  void setNextOffset(long nextOffset) {
+    this.nextOffset = nextOffset;
+  }
 
-  BasicFetchedMessage(long offset, ByteBuffer buffer) {
-    this.offset = offset;
-    this.buffer = buffer;
+  @Override
+  public TopicPartition getTopicPartition() {
+    return topicPartition;
   }
 
   @Override
-  public long getOffset() {
-    return offset;
+  public ByteBuffer getPayload() {
+    return payload;
   }
 
   @Override
-  public ByteBuffer getBuffer() {
-    return buffer;
+  public long getNextOffset() {
+    return nextOffset;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/Bufferer.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/Bufferer.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/Bufferer.java
deleted file mode 100644
index c1fb4f2..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/Bufferer.java
+++ /dev/null
@@ -1,61 +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.twill.internal.kafka.client;
-
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-
-/**
- * A class to help buffering data of format [len][payload-of-len].
- */
-final class Bufferer {
-
-  private ChannelBuffer currentBuffer = null;
-  private int currentSize = -1;
-
-  void apply(ChannelBuffer buffer) {
-    currentBuffer = concatBuffer(currentBuffer, buffer);
-  }
-
-  /**
-   * Returns the buffer if the buffer data is ready to be consumed,
-   * otherwise return {@link ChannelBuffers#EMPTY_BUFFER}.
-   */
-  ChannelBuffer getNext() {
-    if (currentSize < 0) {
-      if (currentBuffer.readableBytes() < 4) {
-        return ChannelBuffers.EMPTY_BUFFER;
-      }
-      currentSize = currentBuffer.readInt();
-    }
-
-    // Keep buffering if less then required number of bytes
-    if (currentBuffer.readableBytes() < currentSize) {
-      return ChannelBuffers.EMPTY_BUFFER;
-    }
-
-    ChannelBuffer result = currentBuffer.readSlice(currentSize);
-    currentSize = -1;
-
-    return result;
-  }
-
-  private ChannelBuffer concatBuffer(ChannelBuffer current, ChannelBuffer buffer) {
-    return current == null ? buffer : ChannelBuffers.wrappedBuffer(current, buffer);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ByteBufferEncoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ByteBufferEncoder.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ByteBufferEncoder.java
new file mode 100644
index 0000000..9211d92
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ByteBufferEncoder.java
@@ -0,0 +1,38 @@
+/*
+ * 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.twill.internal.kafka.client;
+
+import kafka.serializer.Encoder;
+import kafka.utils.VerifiableProperties;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A kafka {@link kafka.serializer.Encoder} for encoding byte buffer into byte array.
+ */
+public final class ByteBufferEncoder implements Encoder<ByteBuffer> {
+
+  public ByteBufferEncoder(VerifiableProperties properties) {
+  }
+
+  public byte[] toBytes(ByteBuffer buffer) {
+    byte[] bytes = new byte[buffer.remaining()];
+    buffer.get(bytes);
+    return bytes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/Compression.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/Compression.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/Compression.java
deleted file mode 100644
index 3355b9f..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/Compression.java
+++ /dev/null
@@ -1,49 +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.twill.internal.kafka.client;
-
-/**
- * Enum for indicating compression method.
- */
-public enum Compression {
-  NONE(0),
-  GZIP(1),
-  SNAPPY(2);
-
-  private final int code;
-
-  Compression(int code) {
-    this.code = code;
-  }
-
-  public int getCode() {
-    return code;
-  }
-
-  public static Compression fromCode(int code) {
-    switch (code) {
-      case 0:
-        return NONE;
-      case 1:
-        return GZIP;
-      case 2:
-        return SNAPPY;
-    }
-    throw new IllegalArgumentException("Unknown compression code.");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ConnectionPool.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ConnectionPool.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ConnectionPool.java
deleted file mode 100644
index c2865ba..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ConnectionPool.java
+++ /dev/null
@@ -1,125 +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.twill.internal.kafka.client;
-
-import com.google.common.collect.Maps;
-import org.jboss.netty.bootstrap.ClientBootstrap;
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelFutureListener;
-import org.jboss.netty.channel.group.ChannelGroup;
-import org.jboss.netty.channel.group.ChannelGroupFuture;
-import org.jboss.netty.channel.group.ChannelGroupFutureListener;
-import org.jboss.netty.channel.group.DefaultChannelGroup;
-
-import java.net.InetSocketAddress;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Provides netty socket connection reuse.
- */
-final class ConnectionPool {
-
-  private final ClientBootstrap bootstrap;
-  private final ChannelGroup channelGroup;
-  private final ConcurrentMap<InetSocketAddress, Queue<ChannelFuture>> connections;
-
-  /**
-   * For releasing a connection back to the pool.
-   */
-  interface ConnectionReleaser {
-    void release();
-  }
-
-  /**
-   * Result of a connect request.
-   */
-  interface ConnectResult extends ConnectionReleaser {
-    ChannelFuture getChannelFuture();
-  }
-
-  ConnectionPool(ClientBootstrap bootstrap) {
-    this.bootstrap = bootstrap;
-    this.channelGroup = new DefaultChannelGroup();
-    this.connections = Maps.newConcurrentMap();
-  }
-
-  ConnectResult connect(InetSocketAddress address) {
-    Queue<ChannelFuture> channelFutures = connections.get(address);
-    if (channelFutures == null) {
-      channelFutures = new ConcurrentLinkedQueue<ChannelFuture>();
-      Queue<ChannelFuture> result = connections.putIfAbsent(address, channelFutures);
-      channelFutures = result == null ? channelFutures : result;
-    }
-
-    ChannelFuture channelFuture = channelFutures.poll();
-    while (channelFuture != null) {
-      if (channelFuture.isSuccess() && channelFuture.getChannel().isConnected()) {
-        return new SimpleConnectResult(address, channelFuture);
-      }
-      channelFuture = channelFutures.poll();
-    }
-
-    channelFuture = bootstrap.connect(address);
-    channelFuture.addListener(new ChannelFutureListener() {
-      @Override
-      public void operationComplete(ChannelFuture future) throws Exception {
-        if (future.isSuccess()) {
-          channelGroup.add(future.getChannel());
-        }
-      }
-    });
-    return new SimpleConnectResult(address, channelFuture);
-  }
-
-  ChannelGroupFuture close() {
-    ChannelGroupFuture result = channelGroup.close();
-    result.addListener(new ChannelGroupFutureListener() {
-      @Override
-      public void operationComplete(ChannelGroupFuture future) throws Exception {
-        bootstrap.releaseExternalResources();
-      }
-    });
-    return result;
-  }
-
-  private final class SimpleConnectResult implements ConnectResult {
-
-    private final InetSocketAddress address;
-    private final ChannelFuture future;
-
-
-    private SimpleConnectResult(InetSocketAddress address, ChannelFuture future) {
-      this.address = address;
-      this.future = future;
-    }
-
-    @Override
-    public ChannelFuture getChannelFuture() {
-      return future;
-    }
-
-    @Override
-    public void release() {
-      if (future.isSuccess()) {
-        connections.get(address).offer(future);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/GZipMessageSetEncoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/GZipMessageSetEncoder.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/GZipMessageSetEncoder.java
deleted file mode 100644
index daa0c2c..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/GZipMessageSetEncoder.java
+++ /dev/null
@@ -1,37 +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.twill.internal.kafka.client;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.zip.GZIPOutputStream;
-
-/**
- * A {@link MessageSetEncoder} that compress message set using GZIP.
- */
-final class GZipMessageSetEncoder extends AbstractCompressedMessageSetEncoder {
-
-  GZipMessageSetEncoder() {
-    super(Compression.GZIP);
-  }
-
-  @Override
-  protected OutputStream createCompressedStream(OutputStream os) throws IOException {
-    return new GZIPOutputStream(os);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IdentityMessageSetEncoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IdentityMessageSetEncoder.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IdentityMessageSetEncoder.java
deleted file mode 100644
index 51dc746..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IdentityMessageSetEncoder.java
+++ /dev/null
@@ -1,42 +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.twill.internal.kafka.client;
-
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-
-/**
- * A pass-through {@link MessageSetEncoder}.
- */
-final class IdentityMessageSetEncoder extends AbstractMessageSetEncoder {
-
-  private ChannelBuffer messageSets = ChannelBuffers.EMPTY_BUFFER;
-
-  @Override
-  public MessageSetEncoder add(ChannelBuffer payload) {
-    messageSets = ChannelBuffers.wrappedBuffer(messageSets, encodePayload(payload));
-    return this;
-  }
-
-  @Override
-  public ChannelBuffer finish() {
-    ChannelBuffer buf = prefixLength(messageSets);
-    messageSets = ChannelBuffers.EMPTY_BUFFER;
-    return buf;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IntegerEncoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IntegerEncoder.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IntegerEncoder.java
new file mode 100644
index 0000000..cbe7eaa
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IntegerEncoder.java
@@ -0,0 +1,35 @@
+/*
+ * 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.twill.internal.kafka.client;
+
+import com.google.common.primitives.Ints;
+import kafka.serializer.Encoder;
+import kafka.utils.VerifiableProperties;
+
+/**
+ * A kafka {@link kafka.serializer.Encoder} for encoding integer into bytes.
+ */
+public final class IntegerEncoder implements Encoder<Integer> {
+
+  public IntegerEncoder(VerifiableProperties properties) {
+  }
+
+  public byte[] toBytes(Integer buffer) {
+    return Ints.toByteArray(buffer.intValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IntegerPartitioner.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IntegerPartitioner.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IntegerPartitioner.java
new file mode 100644
index 0000000..4aa7940
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/IntegerPartitioner.java
@@ -0,0 +1,34 @@
+/*
+ * 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.twill.internal.kafka.client;
+
+import kafka.producer.Partitioner;
+import kafka.utils.VerifiableProperties;
+
+/**
+ * A kafka {@link kafka.producer.Partitioner} using integer key to compute partition id.
+ */
+public final class IntegerPartitioner implements Partitioner<Integer> {
+
+  public IntegerPartitioner(VerifiableProperties properties) {
+  }
+
+  public int partition(Integer key, int numPartitions) {
+    return key % numPartitions;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaBrokerCache.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaBrokerCache.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaBrokerCache.java
deleted file mode 100644
index f2bb815..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaBrokerCache.java
+++ /dev/null
@@ -1,326 +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.twill.internal.kafka.client;
-
-import org.apache.twill.common.Threads;
-import org.apache.twill.zookeeper.NodeChildren;
-import org.apache.twill.zookeeper.NodeData;
-import org.apache.twill.zookeeper.ZKClient;
-import com.google.common.base.Charsets;
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.ImmutableSortedMap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.AbstractIdleService;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetSocketAddress;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.SortedMap;
-
-/**
- * A Service to cache kafka broker information by subscribing to ZooKeeper.
- */
-final class KafkaBrokerCache extends AbstractIdleService {
-
-  private static final Logger LOG = LoggerFactory.getLogger(KafkaBrokerCache.class);
-
-  private static final String BROKERS_PATH = "/brokers";
-
-  private final ZKClient zkClient;
-  private final Map<String, InetSocketAddress> brokers;
-  // topicBrokers is from topic->partition size->brokerId
-  private final Map<String, SortedMap<Integer, Set<String>>> topicBrokers;
-  private final Runnable invokeGetBrokers = new Runnable() {
-    @Override
-    public void run() {
-      getBrokers();
-    }
-  };
-  private final Runnable invokeGetTopics = new Runnable() {
-    @Override
-    public void run() {
-      getTopics();
-    }
-  };
-
-  KafkaBrokerCache(ZKClient zkClient) {
-    this.zkClient = zkClient;
-    this.brokers = Maps.newConcurrentMap();
-    this.topicBrokers = Maps.newConcurrentMap();
-  }
-
-  @Override
-  protected void startUp() throws Exception {
-    getBrokers();
-    getTopics();
-  }
-
-  @Override
-  protected void shutDown() throws Exception {
-    // No-op
-  }
-
-  public int getPartitionSize(String topic) {
-    SortedMap<Integer, Set<String>> partitionBrokers = topicBrokers.get(topic);
-    if (partitionBrokers == null || partitionBrokers.isEmpty()) {
-      return 1;
-    }
-    return partitionBrokers.lastKey();
-  }
-
-  public TopicBroker getBrokerAddress(String topic, int partition) {
-    SortedMap<Integer, Set<String>> partitionBrokers = topicBrokers.get(topic);
-    if (partitionBrokers == null || partitionBrokers.isEmpty()) {
-      return pickRandomBroker(topic);
-    }
-
-    // If the requested partition is greater than supported partition size, randomly pick one
-    if (partition >= partitionBrokers.lastKey()) {
-      return pickRandomBroker(topic);
-    }
-
-    // Randomly pick a partition size and randomly pick a broker from it
-    Random random = new Random();
-    partitionBrokers = partitionBrokers.tailMap(partition + 1);
-    List<Integer> sizes = Lists.newArrayList(partitionBrokers.keySet());
-    Integer partitionSize = pickRandomItem(sizes, random);
-    List<String> ids = Lists.newArrayList(partitionBrokers.get(partitionSize));
-    InetSocketAddress address = brokers.get(ids.get(new Random().nextInt(ids.size())));
-    return address == null ? pickRandomBroker(topic) : new TopicBroker(topic, address, partitionSize);
-  }
-
-  private TopicBroker pickRandomBroker(String topic) {
-    Map.Entry<String, InetSocketAddress> entry = Iterables.getFirst(brokers.entrySet(), null);
-    if (entry == null) {
-      return null;
-    }
-    InetSocketAddress address = entry.getValue();
-    return new TopicBroker(topic, address, 0);
-  }
-
-  private <T> T pickRandomItem(List<T> list, Random random) {
-    return list.get(random.nextInt(list.size()));
-  }
-
-  private void getBrokers() {
-    final String idsPath = BROKERS_PATH + "/ids";
-
-    Futures.addCallback(zkClient.getChildren(idsPath, new Watcher() {
-      @Override
-      public void process(WatchedEvent event) {
-        getBrokers();
-      }
-    }), new ExistsOnFailureFutureCallback<NodeChildren>(idsPath, invokeGetBrokers) {
-      @Override
-      public void onSuccess(NodeChildren result) {
-        Set<String> children = ImmutableSet.copyOf(result.getChildren());
-        for (String child : children) {
-          getBrokenData(idsPath + "/" + child, child);
-        }
-        // Remove all removed brokers
-        removeDiff(children, brokers);
-      }
-    });
-  }
-
-  private void getTopics() {
-    final String topicsPath = BROKERS_PATH + "/topics";
-    Futures.addCallback(zkClient.getChildren(topicsPath, new Watcher() {
-      @Override
-      public void process(WatchedEvent event) {
-        getTopics();
-      }
-    }), new ExistsOnFailureFutureCallback<NodeChildren>(topicsPath, invokeGetTopics) {
-      @Override
-      public void onSuccess(NodeChildren result) {
-        Set<String> children = ImmutableSet.copyOf(result.getChildren());
-
-        // Process new children
-        for (String topic : ImmutableSet.copyOf(Sets.difference(children, topicBrokers.keySet()))) {
-          getTopic(topicsPath + "/" + topic, topic);
-        }
-
-        // Remove old children
-        removeDiff(children, topicBrokers);
-      }
-    });
-  }
-
-  private void getBrokenData(String path, final String brokerId) {
-    Futures.addCallback(zkClient.getData(path), new FutureCallback<NodeData>() {
-      @Override
-      public void onSuccess(NodeData result) {
-        String data = new String(result.getData(), Charsets.UTF_8);
-        String hostPort = data.substring(data.indexOf(':') + 1);
-        int idx = hostPort.indexOf(':');
-        brokers.put(brokerId, new InetSocketAddress(hostPort.substring(0, idx),
-                                                    Integer.parseInt(hostPort.substring(idx + 1))));
-      }
-
-      @Override
-      public void onFailure(Throwable t) {
-        // No-op, the watch on the parent node will handle it.
-      }
-    });
-  }
-
-  private void getTopic(final String path, final String topic) {
-    Futures.addCallback(zkClient.getChildren(path, new Watcher() {
-      @Override
-      public void process(WatchedEvent event) {
-        // Other event type changes are either could be ignored or handled by parent watcher
-        if (event.getType() == Event.EventType.NodeChildrenChanged) {
-          getTopic(path, topic);
-        }
-      }
-    }), new FutureCallback<NodeChildren>() {
-      @Override
-      public void onSuccess(NodeChildren result) {
-        List<String> children = result.getChildren();
-        final List<ListenableFuture<BrokerPartition>> futures = Lists.newArrayListWithCapacity(children.size());
-
-        // Fetch data from each broken node
-        for (final String brokerId : children) {
-          Futures.transform(zkClient.getData(path + "/" + brokerId), new Function<NodeData, BrokerPartition>() {
-            @Override
-            public BrokerPartition apply(NodeData input) {
-              return new BrokerPartition(brokerId, Integer.parseInt(new String(input.getData(), Charsets.UTF_8)));
-            }
-          });
-        }
-
-        // When all fetching is done, build the partition size->broker map for this topic
-        Futures.successfulAsList(futures).addListener(new Runnable() {
-          @Override
-          public void run() {
-            Map<Integer, Set<String>> partitionBrokers = Maps.newHashMap();
-            for (ListenableFuture<BrokerPartition> future : futures) {
-              try {
-                BrokerPartition info = future.get();
-                Set<String> brokerSet = partitionBrokers.get(info.getPartitionSize());
-                if (brokerSet == null) {
-                  brokerSet = Sets.newHashSet();
-                  partitionBrokers.put(info.getPartitionSize(), brokerSet);
-                }
-                brokerSet.add(info.getBrokerId());
-              } catch (Exception e) {
-                // Exception is ignored, as it will be handled by parent watcher
-              }
-            }
-            topicBrokers.put(topic, ImmutableSortedMap.copyOf(partitionBrokers));
-          }
-        }, Threads.SAME_THREAD_EXECUTOR);
-      }
-
-      @Override
-      public void onFailure(Throwable t) {
-        // No-op. Failure would be handled by parent watcher already (e.g. node not exists -> children change in parent)
-      }
-    });
-  }
-
-  private <K, V> void removeDiff(Set<K> keys, Map<K, V> map) {
-    for (K key : ImmutableSet.copyOf(Sets.difference(map.keySet(), keys))) {
-      map.remove(key);
-    }
-  }
-
-  private abstract class ExistsOnFailureFutureCallback<V> implements FutureCallback<V> {
-
-    private final String path;
-    private final Runnable action;
-
-    protected ExistsOnFailureFutureCallback(String path, Runnable action) {
-      this.path = path;
-      this.action = action;
-    }
-
-    @Override
-    public final void onFailure(Throwable t) {
-      if (!isNotExists(t)) {
-        LOG.error("Fail to watch for kafka brokers: " + path, t);
-        return;
-      }
-
-      waitExists(path);
-    }
-
-    private boolean isNotExists(Throwable t) {
-      return ((t instanceof KeeperException) && ((KeeperException) t).code() == KeeperException.Code.NONODE);
-    }
-
-    private void waitExists(String path) {
-      LOG.info("Path " + path + " not exists. Watch for creation.");
-
-      // If the node doesn't exists, use the "exists" call to watch for node creation.
-      Futures.addCallback(zkClient.exists(path, new Watcher() {
-        @Override
-        public void process(WatchedEvent event) {
-          if (event.getType() == Event.EventType.NodeCreated || event.getType() == Event.EventType.NodeDeleted) {
-            action.run();
-          }
-        }
-      }), new FutureCallback<Stat>() {
-        @Override
-        public void onSuccess(Stat result) {
-          // If path exists, get children again, otherwise wait for watch to get triggered
-          if (result != null) {
-            action.run();
-          }
-        }
-        @Override
-        public void onFailure(Throwable t) {
-          action.run();
-        }
-      });
-    }
-  }
-
-  private static final class BrokerPartition {
-    private final String brokerId;
-    private final int partitionSize;
-
-    private BrokerPartition(String brokerId, int partitionSize) {
-      this.brokerId = brokerId;
-      this.partitionSize = partitionSize;
-    }
-
-    public String getBrokerId() {
-      return brokerId;
-    }
-
-    public int getPartitionSize() {
-      return partitionSize;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequest.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequest.java
deleted file mode 100644
index 7b43f8a..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequest.java
+++ /dev/null
@@ -1,91 +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.twill.internal.kafka.client;
-
-import org.jboss.netty.buffer.ChannelBuffer;
-
-/**
- *
- */
-final class KafkaRequest {
-
-  public enum Type {
-    PRODUCE(0),
-    FETCH(1),
-    MULTI_FETCH(2),
-    MULTI_PRODUCE(3),
-    OFFSETS(4);
-
-    private final short id;
-
-    private Type(int id) {
-      this.id = (short) id;
-    }
-
-    public short getId() {
-      return id;
-    }
-  }
-
-  private final Type type;
-  private final String topic;
-  private final int partition;
-  private final ChannelBuffer body;
-  private final ResponseHandler responseHandler;
-
-
-  public static KafkaRequest createProduce(String topic, int partition, ChannelBuffer body) {
-    return new KafkaRequest(Type.PRODUCE, topic, partition, body, ResponseHandler.NO_OP);
-  }
-
-  public static KafkaRequest createFetch(String topic, int partition, ChannelBuffer body, ResponseHandler handler) {
-    return new KafkaRequest(Type.FETCH, topic, partition, body, handler);
-  }
-
-  public static KafkaRequest createOffsets(String topic, int partition, ChannelBuffer body, ResponseHandler handler) {
-    return new KafkaRequest(Type.OFFSETS, topic, partition, body, handler);
-  }
-
-  private KafkaRequest(Type type, String topic, int partition, ChannelBuffer body, ResponseHandler responseHandler) {
-    this.type = type;
-    this.topic = topic;
-    this.partition = partition;
-    this.body = body;
-    this.responseHandler = responseHandler;
-  }
-
-  Type getType() {
-    return type;
-  }
-
-  String getTopic() {
-    return topic;
-  }
-
-  int getPartition() {
-    return partition;
-  }
-
-  ChannelBuffer getBody() {
-    return body;
-  }
-
-  ResponseHandler getResponseHandler() {
-    return responseHandler;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequestEncoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequestEncoder.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequestEncoder.java
deleted file mode 100644
index ef78c76..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequestEncoder.java
+++ /dev/null
@@ -1,60 +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.twill.internal.kafka.client;
-
-import com.google.common.base.Charsets;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.handler.codec.oneone.OneToOneEncoder;
-
-import java.nio.ByteBuffer;
-
-/**
- *
- */
-final class KafkaRequestEncoder extends OneToOneEncoder {
-
-  @Override
-  protected Object encode(ChannelHandlerContext ctx, Channel channel, Object msg) throws Exception {
-    if (!(msg instanceof KafkaRequest)) {
-      return msg;
-    }
-    KafkaRequest req = (KafkaRequest) msg;
-    ByteBuffer topic = Charsets.UTF_8.encode(req.getTopic());
-
-    ChannelBuffer buffer = ChannelBuffers.dynamicBuffer(16 + topic.remaining() + req.getBody().readableBytes());
-    int writerIdx = buffer.writerIndex();
-    buffer.writerIndex(writerIdx + 4);    // Reserves 4 bytes for message length
-
-    // Write out <REQUEST_TYPE>, <TOPIC_LENGTH>, <TOPIC>, <PARTITION>
-    buffer.writeShort(req.getType().getId());
-    buffer.writeShort(topic.remaining());
-    buffer.writeBytes(topic);
-    buffer.writeInt(req.getPartition());
-
-    // Write out the size of the whole buffer (excluding the size field) at the beginning
-    buffer.setInt(writerIdx, buffer.readableBytes() - 4 + req.getBody().readableBytes());
-
-    ChannelBuffer buf = ChannelBuffers.wrappedBuffer(buffer, req.getBody());
-    buf = buf.readBytes(buf.readableBytes());
-
-    return buf;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequestSender.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequestSender.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequestSender.java
deleted file mode 100644
index fbc552c..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaRequestSender.java
+++ /dev/null
@@ -1,26 +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.twill.internal.kafka.client;
-
-/**
- *
- */
-interface KafkaRequestSender {
-
-  void send(KafkaRequest request);
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponse.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponse.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponse.java
deleted file mode 100644
index 68c1bd8..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponse.java
+++ /dev/null
@@ -1,49 +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.twill.internal.kafka.client;
-
-import org.apache.twill.kafka.client.FetchException;
-import org.jboss.netty.buffer.ChannelBuffer;
-
-/**
- *
- */
-final class KafkaResponse {
-
-  private final FetchException.ErrorCode errorCode;
-  private final ChannelBuffer body;
-  private final int size;
-
-  KafkaResponse(FetchException.ErrorCode errorCode, ChannelBuffer body, int size) {
-    this.errorCode = errorCode;
-    this.body = body;
-    this.size = size;
-  }
-
-  public int getSize() {
-    return size;
-  }
-
-  public FetchException.ErrorCode getErrorCode() {
-    return errorCode;
-  }
-
-  public ChannelBuffer getBody() {
-    return body;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponseDispatcher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponseDispatcher.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponseDispatcher.java
deleted file mode 100644
index 47f70ce..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponseDispatcher.java
+++ /dev/null
@@ -1,63 +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.twill.internal.kafka.client;
-
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ExceptionEvent;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelHandler;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.SocketException;
-import java.nio.channels.ClosedChannelException;
-
-/**
- *
- */
-final class KafkaResponseDispatcher extends SimpleChannelHandler {
-
-  private static final Logger LOG = LoggerFactory.getLogger(KafkaResponseDispatcher.class);
-
-  @Override
-  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
-    Object attachment = ctx.getAttachment();
-    if (e.getMessage() instanceof KafkaResponse && attachment instanceof ResponseHandler) {
-      ((ResponseHandler) attachment).received((KafkaResponse) e.getMessage());
-    } else {
-      super.messageReceived(ctx, e);
-    }
-  }
-
-  @Override
-  public void writeRequested(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
-    if (e.getMessage() instanceof KafkaRequest) {
-      ctx.setAttachment(((KafkaRequest) e.getMessage()).getResponseHandler());
-    }
-    super.writeRequested(ctx, e);
-  }
-
-  @Override
-  public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) throws Exception {
-    if (e.getCause() instanceof ClosedChannelException || e.getCause() instanceof SocketException) {
-      // No need to log for socket exception as the client has logic to retry.
-      return;
-    }
-    LOG.warn("Exception caught in kafka client connection.", e.getCause());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponseHandler.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponseHandler.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponseHandler.java
deleted file mode 100644
index 5251e65..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/KafkaResponseHandler.java
+++ /dev/null
@@ -1,51 +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.twill.internal.kafka.client;
-
-import org.apache.twill.kafka.client.FetchException;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelHandler;
-
-/**
- *
- */
-final class KafkaResponseHandler extends SimpleChannelHandler {
-
-  private final Bufferer bufferer = new Bufferer();
-
-  @Override
-  public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
-    Object msg = e.getMessage();
-    if (!(msg instanceof ChannelBuffer)) {
-      super.messageReceived(ctx, e);
-      return;
-    }
-
-    bufferer.apply((ChannelBuffer) msg);
-    ChannelBuffer buffer = bufferer.getNext();
-    while (buffer.readable()) {
-      // Send the response object upstream
-      Channels.fireMessageReceived(ctx, new KafkaResponse(FetchException.ErrorCode.fromCode(buffer.readShort()),
-                                                          buffer, buffer.readableBytes() + 6));
-      buffer = bufferer.getNext();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/MessageFetcher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/MessageFetcher.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/MessageFetcher.java
deleted file mode 100644
index 0814917..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/MessageFetcher.java
+++ /dev/null
@@ -1,243 +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.twill.internal.kafka.client;
-
-import org.apache.twill.common.Threads;
-import org.apache.twill.kafka.client.FetchException;
-import org.apache.twill.kafka.client.FetchedMessage;
-import com.google.common.base.Throwables;
-import com.google.common.collect.AbstractIterator;
-import com.google.common.io.ByteStreams;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBufferInputStream;
-import org.jboss.netty.buffer.ChannelBufferOutputStream;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.xerial.snappy.SnappyInputStream;
-
-import java.io.IOException;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.zip.GZIPInputStream;
-
-/**
- * This class is for consuming messages from a kafka topic.
- */
-final class MessageFetcher extends AbstractIterator<FetchedMessage> implements ResponseHandler {
-
-  private static final long BACKOFF_INTERVAL_MS = 100;
-
-  private final KafkaRequestSender sender;
-  private final String topic;
-  private final int partition;
-  private final int maxSize;
-  private final AtomicLong offset;
-  private final BlockingQueue<FetchResult> messages;
-  private final ScheduledExecutorService scheduler;
-  private volatile long backoffMillis;
-  private final Runnable sendFetchRequest = new Runnable() {
-    @Override
-    public void run() {
-      sendFetchRequest();
-    }
-  };
-
-  MessageFetcher(String topic, int partition, long offset, int maxSize, KafkaRequestSender sender) {
-    this.topic = topic;
-    this.partition = partition;
-    this.sender = sender;
-    this.offset = new AtomicLong(offset);
-    this.maxSize = maxSize;
-    this.messages = new LinkedBlockingQueue<FetchResult>();
-    this.scheduler = Executors.newSingleThreadScheduledExecutor(
-                        Threads.createDaemonThreadFactory("kafka-" + topic + "-consumer"));
-  }
-
-  @Override
-  public void received(KafkaResponse response) {
-    if (response.getErrorCode() != FetchException.ErrorCode.OK) {
-      messages.add(FetchResult.failure(new FetchException("Error in fetching: " + response.getErrorCode(),
-                                                          response.getErrorCode())));
-      return;
-    }
-
-    try {
-      if (decodeResponse(response.getBody(), -1)) {
-        backoffMillis = 0;
-      } else {
-        backoffMillis = Math.max(backoffMillis + BACKOFF_INTERVAL_MS, 1000);
-        scheduler.schedule(sendFetchRequest, backoffMillis, TimeUnit.MILLISECONDS);
-      }
-    } catch (Throwable t) {
-      messages.add(FetchResult.failure(t));
-    }
-  }
-
-  private boolean decodeResponse(ChannelBuffer buffer, long nextOffset) {
-    boolean hasMessage = false;
-    boolean computeOffset = nextOffset < 0;
-    while (buffer.readableBytes() >= 4) {
-      int size = buffer.readInt();
-      if (buffer.readableBytes() < size) {
-        if (!hasMessage) {
-          throw new IllegalStateException("Size too small");
-        }
-        break;
-      }
-      nextOffset = computeOffset ? offset.addAndGet(size + 4) : nextOffset;
-      decodeMessage(size, buffer, nextOffset);
-      hasMessage = true;
-    }
-    return hasMessage;
-
-  }
-
-  private void decodeMessage(int size, ChannelBuffer buffer, long nextOffset) {
-    int readerIdx = buffer.readerIndex();
-    int magic = buffer.readByte();
-    Compression compression = magic == 0 ? Compression.NONE : Compression.fromCode(buffer.readByte());
-    int crc = buffer.readInt();
-
-    ChannelBuffer payload = buffer.readSlice(size - (buffer.readerIndex() - readerIdx));
-
-    // Verify CRC?
-    enqueueMessage(compression, payload, nextOffset);
-  }
-
-  private void enqueueMessage(Compression compression, ChannelBuffer payload, long nextOffset) {
-    switch (compression) {
-      case NONE:
-        messages.add(FetchResult.success(new BasicFetchedMessage(nextOffset, payload.toByteBuffer())));
-        break;
-      case GZIP:
-        decodeResponse(gunzip(payload), nextOffset);
-        break;
-      case SNAPPY:
-        decodeResponse(unsnappy(payload), nextOffset);
-        break;
-    }
-  }
-
-  private ChannelBuffer gunzip(ChannelBuffer source) {
-    ChannelBufferOutputStream output = new ChannelBufferOutputStream(
-                                              ChannelBuffers.dynamicBuffer(source.readableBytes() * 2));
-    try {
-      try {
-        GZIPInputStream gzipInput = new GZIPInputStream(new ChannelBufferInputStream(source));
-        try {
-          ByteStreams.copy(gzipInput, output);
-          return output.buffer();
-        } finally {
-          gzipInput.close();
-        }
-      } finally {
-        output.close();
-      }
-    } catch (IOException e) {
-      throw Throwables.propagate(e);
-    }
-  }
-
-  private ChannelBuffer unsnappy(ChannelBuffer source) {
-    ChannelBufferOutputStream output = new ChannelBufferOutputStream(
-                                              ChannelBuffers.dynamicBuffer(source.readableBytes() * 2));
-    try {
-      try {
-        SnappyInputStream snappyInput = new SnappyInputStream(new ChannelBufferInputStream(source));
-        try {
-          ByteStreams.copy(snappyInput, output);
-          return output.buffer();
-        } finally {
-          snappyInput.close();
-        }
-      } finally {
-        output.close();
-      }
-    } catch (IOException e) {
-      throw Throwables.propagate(e);
-    }
-  }
-
-  private void sendFetchRequest() {
-    ChannelBuffer fetchBody = ChannelBuffers.buffer(12);
-    fetchBody.writeLong(offset.get());
-    fetchBody.writeInt(maxSize);
-    sender.send(KafkaRequest.createFetch(topic, partition, fetchBody, MessageFetcher.this));
-  }
-
-  @Override
-  protected FetchedMessage computeNext() {
-    FetchResult result = messages.poll();
-    if (result != null) {
-      return getMessage(result);
-    }
-
-    try {
-      sendFetchRequest();
-      return getMessage(messages.take());
-    } catch (InterruptedException e) {
-      scheduler.shutdownNow();
-      return endOfData();
-    }
-  }
-
-  private FetchedMessage getMessage(FetchResult result) {
-    try {
-      if (result.isSuccess()) {
-        return result.getMessage();
-      } else {
-        throw result.getErrorCause();
-      }
-    } catch (Throwable t) {
-      throw Throwables.propagate(t);
-    }
-  }
-
-  private static final class FetchResult {
-    private final FetchedMessage message;
-    private final Throwable errorCause;
-
-    static FetchResult success(FetchedMessage message) {
-      return new FetchResult(message, null);
-    }
-
-    static FetchResult failure(Throwable cause) {
-      return new FetchResult(null, cause);
-    }
-
-    private FetchResult(FetchedMessage message, Throwable errorCause) {
-      this.message = message;
-      this.errorCause = errorCause;
-    }
-
-    public FetchedMessage getMessage() {
-      return message;
-    }
-
-    public Throwable getErrorCause() {
-      return errorCause;
-    }
-
-    public boolean isSuccess() {
-      return message != null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/MessageSetEncoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/MessageSetEncoder.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/MessageSetEncoder.java
deleted file mode 100644
index 49008cc..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/MessageSetEncoder.java
+++ /dev/null
@@ -1,31 +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.twill.internal.kafka.client;
-
-import org.jboss.netty.buffer.ChannelBuffer;
-
-/**
- * This represents a set of messages that goes into the same message set and get encoded as
- * single kafka message set.
- */
-interface MessageSetEncoder {
-
-  MessageSetEncoder add(ChannelBuffer payload);
-
-  ChannelBuffer finish();
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fcbe54d5/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ResponseHandler.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ResponseHandler.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ResponseHandler.java
deleted file mode 100644
index f681b85..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/ResponseHandler.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.twill.internal.kafka.client;
-
-/**
- * Represents handler for kafka response.
- */
-interface ResponseHandler {
-
-  ResponseHandler NO_OP = new ResponseHandler() {
-    @Override
-    public void received(KafkaResponse response) {
-      // No-op
-    }
-  };
-
-  void received(KafkaResponse response);
-}


[50/50] [abbrv] git commit: Enable stricter checkstyle rule for whitespace around LCURLY

Posted by ch...@apache.org.
Enable stricter checkstyle rule for whitespace around LCURLY

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

Branch: refs/heads/site
Commit: 2e52a947cee9ec4662e24e9d099b43f0f35d3de8
Parents: 9393df8
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Apr 21 11:10:28 2014 -0700
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Apr 21 11:10:28 2014 -0700

----------------------------------------------------------------------
 checkstyle.xml                                               | 4 ++--
 .../java/org/apache/twill/api/ResourceSpecification.java     | 8 ++++----
 .../twill/internal/json/ResourceSpecificationCodecTest.java  | 2 +-
 .../java/org/apache/twill/internal/yarn/YarnAMClient.java    | 2 +-
 4 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/2e52a947/checkstyle.xml
----------------------------------------------------------------------
diff --git a/checkstyle.xml b/checkstyle.xml
index 6404c8f..8729af2 100644
--- a/checkstyle.xml
+++ b/checkstyle.xml
@@ -326,11 +326,11 @@ page at http://checkstyle.sourceforge.net/config.html -->
       -->
       <property name="tokens" value="ASSIGN, BAND, BAND_ASSIGN, BOR,
         BOR_ASSIGN, BSR, BSR_ASSIGN, BXOR, BXOR_ASSIGN, COLON, DIV, DIV_ASSIGN,
-        EQUAL, GE, GT, LAND, LCURLY, LE, LITERAL_CATCH, LITERAL_DO, LITERAL_ELSE,
+        EQUAL, GE, GT, LAND, LE, LITERAL_CATCH, LITERAL_DO, LITERAL_ELSE,
         LITERAL_FINALLY, LITERAL_FOR, LITERAL_IF, LITERAL_RETURN,
         LITERAL_SYNCHRONIZED, LITERAL_TRY, LITERAL_WHILE, LOR, LT, MINUS,
         MINUS_ASSIGN, MOD, MOD_ASSIGN, NOT_EQUAL, PLUS, PLUS_ASSIGN, QUESTION,
-        SL, SL_ASSIGN, SR_ASSIGN, STAR, STAR_ASSIGN"/>
+        SL, SLIST, SL_ASSIGN, SR_ASSIGN, STAR, STAR_ASSIGN"/>
       <property name="allowEmptyConstructors" value="true"/>
       <property name="allowEmptyMethods" value="true"/>
       <property name="severity" value="error"/>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/2e52a947/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
index 7600041..4b602bc 100644
--- a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
@@ -164,8 +164,8 @@ public interface ResourceSpecification {
         return new Done();
       }
 
-      public Done setRacks(Iterable<String> racks){
-        if (racks != null){
+      public Done setRacks(Iterable<String> racks) {
+        if (racks != null) {
           Iterables.addAll(Builder.this.racks, racks);
         }
         return new Done();
@@ -180,8 +180,8 @@ public interface ResourceSpecification {
         return new AfterHosts();
       }
 
-      public AfterHosts setHosts(Iterable<String> hosts){
-        if (hosts != null){
+      public AfterHosts setHosts(Iterable<String> hosts) {
+        if (hosts != null) {
           Iterables.addAll(Builder.this.hosts, hosts);
         }
         return new AfterHosts();

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/2e52a947/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java b/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
index 96f272c..e8b0eff 100644
--- a/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
+++ b/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
@@ -29,7 +29,7 @@ import org.unitils.reflectionassert.ReflectionAssert;
 import java.util.Arrays;
 
 /**
- * Maybe this checkstyle rule needs to be removed
+ * Maybe this checkstyle rule needs to be removed.
  */
 public class ResourceSpecificationCodecTest {
   private final Gson gson = new GsonBuilder().serializeNulls()

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/2e52a947/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
index 6a5ee36..a5a061a 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
@@ -72,7 +72,7 @@ public interface YarnAMClient extends Service {
     public abstract String apply();
 
     private <T> ContainerRequestBuilder add(Collection<T> collection, Collection<T> more) {
-      if (more != null){
+      if (more != null) {
         collection.addAll(more);
       }
       return this;


[34/50] [abbrv] git commit: Added missing license header.

Posted by ch...@apache.org.
Added missing license header.

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

Branch: refs/heads/site
Commit: a96dbdc83d0b81e265a8ec029a1cb91eb2a01f7f
Parents: 647880e
Author: Terence Yim <te...@continuuity.com>
Authored: Wed Feb 26 15:12:43 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Wed Feb 26 15:12:43 2014 -0800

----------------------------------------------------------------------
 .../json/ResourceSpecificationCodecTest.java       | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/a96dbdc8/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java b/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
index 044a465..96f272c 100644
--- a/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
+++ b/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
@@ -1,3 +1,20 @@
+/*
+ * 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.twill.internal.json;
 
 import com.google.gson.Gson;


[27/50] [abbrv] git commit: (TWILL-45) Included Throwable information in LogEntry

Posted by ch...@apache.org.
(TWILL-45) Included Throwable information in LogEntry

1. API change in LogEntry class
  - Add getThrowable() method to return LogThrowable
  - Deprecated getStackTraces(). Implementation is to lookup stack trace from the LogThrowable.
2. Refactor Gson encode/decode of Log entry related classes to simplify logic.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: e5c62e5cdfc90d10cff166acb548352cbb4347ae
Parents: 146740b
Author: Terence Yim <te...@continuuity.com>
Authored: Tue Feb 11 15:22:02 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Fri Feb 14 00:21:04 2014 -0800

----------------------------------------------------------------------
 .../org/apache/twill/api/logging/LogEntry.java  |  44 +++++++
 .../apache/twill/api/logging/LogThrowable.java  |  48 +++++++
 .../twill/api/logging/PrinterLogHandler.java    |  25 +++-
 .../twill/internal/AbstractTwillController.java |   8 +-
 .../internal/json/DefaultLogThrowable.java      |  66 ++++++++++
 .../internal/json/ILoggingEventSerializer.java  |  71 +++++++++++
 .../twill/internal/json/LogEntryDecoder.java    | 124 +++++++++++++++++++
 .../twill/internal/json/LogThrowableCodec.java  |  56 +++++++++
 .../twill/internal/logging/KafkaAppender.java   |  85 +++----------
 .../twill/internal/logging/LogEntryDecoder.java | 124 -------------------
 .../apache/twill/yarn/LogHandlerTestRun.java    |  32 ++++-
 11 files changed, 483 insertions(+), 200 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e5c62e5c/twill-api/src/main/java/org/apache/twill/api/logging/LogEntry.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/logging/LogEntry.java b/twill-api/src/main/java/org/apache/twill/api/logging/LogEntry.java
index 4995328..eb1bdf8 100644
--- a/twill-api/src/main/java/org/apache/twill/api/logging/LogEntry.java
+++ b/twill-api/src/main/java/org/apache/twill/api/logging/LogEntry.java
@@ -34,25 +34,69 @@ public interface LogEntry {
     TRACE
   }
 
+  /**
+   * Returns name of the logger.
+   */
   String getLoggerName();
 
+  /**
+   * Returns hostname of where the log emitted.
+   */
   String getHost();
 
+  /**
+   * Returns timestamp of the log.
+   */
   long getTimestamp();
 
+  /**
+   * Returns the log {@link Level} of the log.
+   */
   Level getLogLevel();
 
+  /**
+   * Returns the class name where the log emitted.
+   */
   String getSourceClassName();
 
+  /**
+   * Returns the method name where the log emitted.
+   */
   String getSourceMethodName();
 
+  /**
+   * Returns the source file name where the log emitted.
+   */
   String getFileName();
 
+  /**
+   * Returns the line number in the source file where the log emitted.
+   */
   int getLineNumber();
 
+  /**
+   * Returns the name of the thread where the log emitted.
+   */
   String getThreadName();
 
+  /**
+   * Returns the log message.
+   */
   String getMessage();
 
+  /**
+   * Returns the {@link Throwable} information emitted with the log.
+   *
+   * @return A {@link LogThrowable} or {@code null} if {@link Throwable} information is not available.
+   */
+  LogThrowable getThrowable();
+
+  /**
+   * Returns the stack trace of the throwable information emitted with the log.
+   *
+   * @return the stack trace information or an empty array if {@link Throwable} information is not available.
+   * @deprecated Use {@link #getThrowable()} instead.
+   */
+  @Deprecated
   StackTraceElement[] getStackTraces();
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e5c62e5c/twill-api/src/main/java/org/apache/twill/api/logging/LogThrowable.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/logging/LogThrowable.java b/twill-api/src/main/java/org/apache/twill/api/logging/LogThrowable.java
new file mode 100644
index 0000000..c25cd75
--- /dev/null
+++ b/twill-api/src/main/java/org/apache/twill/api/logging/LogThrowable.java
@@ -0,0 +1,48 @@
+/*
+ * 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.twill.api.logging;
+
+/**
+ * Carries {@link Throwable} information in a {@link LogEntry}.
+ */
+public interface LogThrowable {
+
+  /**
+   * Returns the name of the Throwable class.
+   */
+  String getClassName();
+
+  /**
+   * Returns the message contained inside the Throwable.
+   *
+   * @return A {@link String} message or {@code null} if such message is not available.
+   */
+  String getMessage();
+
+  /**
+   * Returns the stack trace of the Throwable.
+   */
+  StackTraceElement[] getStackTraces();
+
+  /**
+   * Returns the cause of this {@link LogThrowable}.
+   *
+   * @return The {@link LogThrowable} cause or {@code null} if no cause is available.
+   */
+  LogThrowable getCause();
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e5c62e5c/twill-api/src/main/java/org/apache/twill/api/logging/PrinterLogHandler.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/logging/PrinterLogHandler.java b/twill-api/src/main/java/org/apache/twill/api/logging/PrinterLogHandler.java
index 71a2bca..c6d6501 100644
--- a/twill-api/src/main/java/org/apache/twill/api/logging/PrinterLogHandler.java
+++ b/twill-api/src/main/java/org/apache/twill/api/logging/PrinterLogHandler.java
@@ -69,14 +69,27 @@ public final class PrinterLogHandler implements LogHandler {
                      logEntry.getMessage());
     formatter.flush();
 
-    StackTraceElement[] stackTraces = logEntry.getStackTraces();
-    if (stackTraces != null) {
-      for (StackTraceElement stackTrace : stackTraces) {
-        writer.append("\tat ").append(stackTrace.toString());
-        writer.println();
+    // Prints the throwable and stack trace.
+    LogThrowable throwable = logEntry.getThrowable();
+    while (throwable != null) {
+      writer.append(throwable.getClassName()).append(": ").append(throwable.getMessage());
+      writer.println();
+
+      StackTraceElement[] stackTraces = throwable.getStackTraces();
+      if (stackTraces != null) {
+        for (StackTraceElement stackTrace : stackTraces) {
+          writer.append("\tat ").append(stackTrace.toString());
+          writer.println();
+        }
+      }
+
+      throwable = throwable.getCause();
+      if (throwable != null) {
+        writer.append("Caused by: ");
       }
-      writer.flush();
     }
+
+    writer.flush();
   }
 
   private String timestampToUTC(long timestamp) {

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e5c62e5c/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java b/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
index d45a7c3..bf20616 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/AbstractTwillController.java
@@ -26,13 +26,15 @@ import org.apache.twill.api.RunId;
 import org.apache.twill.api.TwillController;
 import org.apache.twill.api.logging.LogEntry;
 import org.apache.twill.api.logging.LogHandler;
+import org.apache.twill.api.logging.LogThrowable;
 import org.apache.twill.common.Cancellable;
 import org.apache.twill.discovery.DiscoveryServiceClient;
 import org.apache.twill.discovery.ServiceDiscovered;
 import org.apache.twill.discovery.ZKDiscoveryService;
+import org.apache.twill.internal.json.LogEntryDecoder;
+import org.apache.twill.internal.json.LogThrowableCodec;
 import org.apache.twill.internal.json.StackTraceElementCodec;
 import org.apache.twill.internal.kafka.client.ZKKafkaClientService;
-import org.apache.twill.internal.logging.LogEntryDecoder;
 import org.apache.twill.internal.state.SystemMessages;
 import org.apache.twill.kafka.client.FetchedMessage;
 import org.apache.twill.kafka.client.KafkaClientService;
@@ -109,7 +111,9 @@ public abstract class AbstractTwillController extends AbstractZKServiceControlle
 
   private static final class LogMessageCallback implements KafkaConsumer.MessageCallback {
 
-    private static final Gson GSON = new GsonBuilder().registerTypeAdapter(LogEntry.class, new LogEntryDecoder())
+    private static final Gson GSON = new GsonBuilder()
+      .registerTypeAdapter(LogEntry.class, new LogEntryDecoder())
+      .registerTypeAdapter(LogThrowable.class, new LogThrowableCodec())
       .registerTypeAdapter(StackTraceElement.class, new StackTraceElementCodec())
       .create();
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e5c62e5c/twill-core/src/main/java/org/apache/twill/internal/json/DefaultLogThrowable.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/DefaultLogThrowable.java b/twill-core/src/main/java/org/apache/twill/internal/json/DefaultLogThrowable.java
new file mode 100644
index 0000000..cd69bb9
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/DefaultLogThrowable.java
@@ -0,0 +1,66 @@
+/*
+ * 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.twill.internal.json;
+
+import ch.qos.logback.classic.spi.IThrowableProxy;
+import ch.qos.logback.classic.spi.StackTraceElementProxy;
+import org.apache.twill.api.logging.LogThrowable;
+
+/**
+ * Default implementation of the {@link LogThrowable} interface.
+ */
+final class DefaultLogThrowable implements LogThrowable {
+
+  private String className;
+  private String message;
+  private StackTraceElement[] stackTraces;
+  private LogThrowable cause;
+
+  DefaultLogThrowable(IThrowableProxy throwableProxy) {
+    this.className = throwableProxy.getClassName();
+    this.message = throwableProxy.getMessage();
+
+    StackTraceElementProxy[] stackTraceElementProxyArray = throwableProxy.getStackTraceElementProxyArray();
+    this.stackTraces = new StackTraceElement[stackTraceElementProxyArray.length];
+    for (int i = 0; i < stackTraceElementProxyArray.length; i++) {
+      stackTraces[i] = stackTraceElementProxyArray[i].getStackTraceElement();
+    }
+
+    cause = (throwableProxy.getCause() == null) ? null : new DefaultLogThrowable(throwableProxy.getCause());
+  }
+
+  @Override
+  public String getClassName() {
+    return className;
+  }
+
+  @Override
+  public String getMessage() {
+    return message;
+  }
+
+  @Override
+  public StackTraceElement[] getStackTraces() {
+    return stackTraces;
+  }
+
+  @Override
+  public LogThrowable getCause() {
+    return cause;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e5c62e5c/twill-core/src/main/java/org/apache/twill/internal/json/ILoggingEventSerializer.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/ILoggingEventSerializer.java b/twill-core/src/main/java/org/apache/twill/internal/json/ILoggingEventSerializer.java
new file mode 100644
index 0000000..a3c7add
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/ILoggingEventSerializer.java
@@ -0,0 +1,71 @@
+/*
+ * 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.twill.internal.json;
+
+import ch.qos.logback.classic.pattern.ClassOfCallerConverter;
+import ch.qos.logback.classic.pattern.FileOfCallerConverter;
+import ch.qos.logback.classic.pattern.LineOfCallerConverter;
+import ch.qos.logback.classic.pattern.MethodOfCallerConverter;
+import ch.qos.logback.classic.spi.ILoggingEvent;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonNull;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
+import org.apache.twill.api.logging.LogThrowable;
+
+import java.lang.reflect.Type;
+
+/**
+ * Gson serializer for {@link ILoggingEvent}.
+ */
+public final class ILoggingEventSerializer implements JsonSerializer<ILoggingEvent> {
+
+  private final ClassOfCallerConverter classNameConverter = new ClassOfCallerConverter();
+  private final MethodOfCallerConverter methodConverter = new MethodOfCallerConverter();
+  private final FileOfCallerConverter fileConverter = new FileOfCallerConverter();
+  private final LineOfCallerConverter lineConverter = new LineOfCallerConverter();
+  private final String hostname;
+
+  public ILoggingEventSerializer(String hostname) {
+    this.hostname = hostname;
+  }
+
+  @Override
+  public JsonElement serialize(ILoggingEvent event, Type typeOfSrc, JsonSerializationContext context) {
+    JsonObject json = new JsonObject();
+    json.addProperty("name", event.getLoggerName());
+    json.addProperty("host", hostname);
+    json.addProperty("timestamp", Long.toString(event.getTimeStamp()));
+    json.addProperty("level", event.getLevel().toString());
+    json.addProperty("className", classNameConverter.convert(event));
+    json.addProperty("method", methodConverter.convert(event));
+    json.addProperty("file", fileConverter.convert(event));
+    json.addProperty("line", lineConverter.convert(event));
+    json.addProperty("thread", event.getThreadName());
+    json.addProperty("message", event.getFormattedMessage());
+
+    if (event.getThrowableProxy() == null) {
+      json.add("throwable", JsonNull.INSTANCE);
+    } else {
+      json.add("throwable", context.serialize(new DefaultLogThrowable(event.getThrowableProxy()), LogThrowable.class));
+    }
+
+    return json;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e5c62e5c/twill-core/src/main/java/org/apache/twill/internal/json/LogEntryDecoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/LogEntryDecoder.java b/twill-core/src/main/java/org/apache/twill/internal/json/LogEntryDecoder.java
new file mode 100644
index 0000000..b047648
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/LogEntryDecoder.java
@@ -0,0 +1,124 @@
+/*
+ * 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.twill.internal.json;
+
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParseException;
+import org.apache.twill.api.logging.LogEntry;
+import org.apache.twill.api.logging.LogThrowable;
+
+import java.lang.reflect.Type;
+
+/**
+ * A Gson decoder for {@link LogEntry}.
+ */
+public final class LogEntryDecoder implements JsonDeserializer<LogEntry> {
+
+  private static final StackTraceElement[] EMPTY_STACK_TRACES = new StackTraceElement[0];
+
+  @Override
+  public LogEntry deserialize(JsonElement json, Type typeOfT,
+                              JsonDeserializationContext context) throws JsonParseException {
+    if (!json.isJsonObject()) {
+      return null;
+    }
+    JsonObject jsonObj = json.getAsJsonObject();
+
+    final String name = JsonUtils.getAsString(jsonObj, "name");
+    final String host = JsonUtils.getAsString(jsonObj, "host");
+    final long timestamp = JsonUtils.getAsLong(jsonObj, "timestamp", 0);
+    final LogEntry.Level logLevel = LogEntry.Level.valueOf(JsonUtils.getAsString(jsonObj, "level"));
+    final String className = JsonUtils.getAsString(jsonObj, "className");
+    final String method = JsonUtils.getAsString(jsonObj, "method");
+    final String file = JsonUtils.getAsString(jsonObj, "file");
+    final String line = JsonUtils.getAsString(jsonObj, "line");
+    final String thread = JsonUtils.getAsString(jsonObj, "thread");
+    final String message = JsonUtils.getAsString(jsonObj, "message");
+    final LogThrowable logThrowable = context.deserialize(jsonObj.get("throwable"), LogThrowable.class);
+
+    return new LogEntry() {
+      @Override
+      public String getLoggerName() {
+        return name;
+      }
+
+      @Override
+      public String getHost() {
+        return host;
+      }
+
+      @Override
+      public long getTimestamp() {
+        return timestamp;
+      }
+
+      @Override
+      public Level getLogLevel() {
+        return logLevel;
+      }
+
+      @Override
+      public String getSourceClassName() {
+        return className;
+      }
+
+      @Override
+      public String getSourceMethodName() {
+        return method;
+      }
+
+      @Override
+      public String getFileName() {
+        return file;
+      }
+
+      @Override
+      public int getLineNumber() {
+        if (line.equals("?")) {
+          return -1;
+        } else {
+          return Integer.parseInt(line);
+        }
+      }
+
+      @Override
+      public String getThreadName() {
+        return thread;
+      }
+
+      @Override
+      public String getMessage() {
+        return message;
+      }
+
+      @Override
+      public LogThrowable getThrowable() {
+        return logThrowable;
+      }
+
+      @Override
+      public StackTraceElement[] getStackTraces() {
+        LogThrowable throwable = getThrowable();
+        return (throwable == null) ? EMPTY_STACK_TRACES : throwable.getStackTraces();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e5c62e5c/twill-core/src/main/java/org/apache/twill/internal/json/LogThrowableCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/LogThrowableCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/LogThrowableCodec.java
new file mode 100644
index 0000000..d159d42
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/LogThrowableCodec.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.twill.internal.json;
+
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParseException;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
+import org.apache.twill.api.logging.LogThrowable;
+
+import java.lang.reflect.Type;
+
+/**
+ * Gson codec for {@link LogThrowable}.
+ */
+public final class LogThrowableCodec implements JsonSerializer<LogThrowable>, JsonDeserializer<LogThrowable> {
+
+  @Override
+  public JsonElement serialize(LogThrowable throwable, Type typeOfSrc, JsonSerializationContext context) {
+    JsonObject json = new JsonObject();
+    json.addProperty("className", throwable.getClassName());
+    json.addProperty("message", throwable.getMessage());
+    json.add("stackTraces", context.serialize(throwable.getStackTraces(), StackTraceElement[].class));
+
+    LogThrowable cause = throwable.getCause();
+    if (cause != null) {
+      json.add("cause", context.serialize(cause, LogThrowable.class));
+    }
+
+    return json;
+  }
+
+  @Override
+  public LogThrowable deserialize(JsonElement json, Type typeOfT,
+                                  JsonDeserializationContext context) throws JsonParseException {
+    return context.deserialize(json, DefaultLogThrowable.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e5c62e5c/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java b/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
index 8345865..f344773 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
@@ -17,27 +17,25 @@
  */
 package org.apache.twill.internal.logging;
 
-import ch.qos.logback.classic.pattern.ClassOfCallerConverter;
-import ch.qos.logback.classic.pattern.FileOfCallerConverter;
-import ch.qos.logback.classic.pattern.LineOfCallerConverter;
-import ch.qos.logback.classic.pattern.MethodOfCallerConverter;
 import ch.qos.logback.classic.spi.ILoggingEvent;
-import ch.qos.logback.classic.spi.IThrowableProxy;
-import ch.qos.logback.classic.spi.StackTraceElementProxy;
 import ch.qos.logback.core.AppenderBase;
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
-import com.google.common.base.Throwables;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Service;
-import com.google.gson.stream.JsonWriter;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.twill.api.logging.LogThrowable;
 import org.apache.twill.common.Services;
 import org.apache.twill.common.Threads;
+import org.apache.twill.internal.json.ILoggingEventSerializer;
+import org.apache.twill.internal.json.LogThrowableCodec;
+import org.apache.twill.internal.json.StackTraceElementCodec;
 import org.apache.twill.internal.kafka.client.ZKKafkaClientService;
 import org.apache.twill.kafka.client.Compression;
 import org.apache.twill.kafka.client.KafkaClientService;
@@ -49,8 +47,6 @@ import org.apache.twill.zookeeper.ZKClients;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.io.StringWriter;
 import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.List;
@@ -71,7 +67,6 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
 
   private static final Logger LOG = LoggerFactory.getLogger(KafkaAppender.class);
 
-  private final LogEventConverter eventConverter;
   private final AtomicReference<KafkaPublisher.Preparer> publisher;
   private final Runnable flushTask;
   /**
@@ -79,6 +74,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
    */
   private final AtomicInteger bufferedSize;
 
+  private LogEventConverter eventConverter;
   private ZKClientService zkClientService;
   private KafkaClientService kafkaClient;
   private String zkConnectStr;
@@ -90,7 +86,6 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
   private ScheduledExecutorService scheduler;
 
   public KafkaAppender() {
-    eventConverter = new LogEventConverter();
     publisher = new AtomicReference<KafkaPublisher.Preparer>();
     flushTask = createFlushTask();
     bufferedSize = new AtomicInteger();
@@ -141,6 +136,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
   public void start() {
     Preconditions.checkNotNull(zkConnectStr);
 
+    eventConverter = new LogEventConverter(hostname);
     scheduler = Executors.newSingleThreadScheduledExecutor(Threads.createDaemonThreadFactory("kafka-logger"));
 
     zkClientService = ZKClientServices.delegate(
@@ -286,65 +282,20 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
   /**
    * Helper class to convert {@link ILoggingEvent} into json string.
    */
-  private final class LogEventConverter {
+  private static final class LogEventConverter {
 
-    private final ClassOfCallerConverter classNameConverter = new ClassOfCallerConverter();
-    private final MethodOfCallerConverter methodConverter = new MethodOfCallerConverter();
-    private final FileOfCallerConverter fileConverter = new FileOfCallerConverter();
-    private final LineOfCallerConverter lineConverter = new LineOfCallerConverter();
+    private final Gson gson;
 
-    private String convert(ILoggingEvent event) {
-      StringWriter result = new StringWriter();
-      JsonWriter writer = new JsonWriter(result);
-
-      try {
-        try {
-          writer.beginObject();
-          writer.name("name").value(event.getLoggerName());
-          writer.name("host").value(hostname);
-          writer.name("timestamp").value(Long.toString(event.getTimeStamp()));
-          writer.name("level").value(event.getLevel().toString());
-          writer.name("className").value(classNameConverter.convert(event));
-          writer.name("method").value(methodConverter.convert(event));
-          writer.name("file").value(fileConverter.convert(event));
-          writer.name("line").value(lineConverter.convert(event));
-          writer.name("thread").value(event.getThreadName());
-          writer.name("message").value(event.getFormattedMessage());
-          writer.name("stackTraces");
-          encodeStackTraces(event.getThrowableProxy(), writer);
-
-          writer.endObject();
-        } finally {
-          writer.close();
-        }
-      } catch (IOException e) {
-        throw Throwables.propagate(e);
-      }
-
-      return result.toString();
+    private LogEventConverter(String hostname) {
+      gson = new GsonBuilder()
+        .registerTypeAdapter(StackTraceElement.class, new StackTraceElementCodec())
+        .registerTypeAdapter(LogThrowable.class, new LogThrowableCodec())
+        .registerTypeAdapter(ILoggingEvent.class, new ILoggingEventSerializer(hostname))
+        .create();
     }
 
-    private void encodeStackTraces(IThrowableProxy throwable, JsonWriter writer) throws IOException {
-      writer.beginArray();
-      try {
-        if (throwable == null) {
-          return;
-        }
-
-        for (StackTraceElementProxy stackTrace : throwable.getStackTraceElementProxyArray()) {
-          writer.beginObject();
-
-          StackTraceElement element = stackTrace.getStackTraceElement();
-          writer.name("className").value(element.getClassName());
-          writer.name("method").value(element.getMethodName());
-          writer.name("file").value(element.getFileName());
-          writer.name("line").value(element.getLineNumber());
-
-          writer.endObject();
-        }
-      } finally {
-        writer.endArray();
-      }
+    private String convert(ILoggingEvent event) {
+      return gson.toJson(event, ILoggingEvent.class);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e5c62e5c/twill-core/src/main/java/org/apache/twill/internal/logging/LogEntryDecoder.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/logging/LogEntryDecoder.java b/twill-core/src/main/java/org/apache/twill/internal/logging/LogEntryDecoder.java
deleted file mode 100644
index 9bb0896..0000000
--- a/twill-core/src/main/java/org/apache/twill/internal/logging/LogEntryDecoder.java
+++ /dev/null
@@ -1,124 +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.twill.internal.logging;
-
-import com.google.gson.JsonDeserializationContext;
-import com.google.gson.JsonDeserializer;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonObject;
-import com.google.gson.JsonParseException;
-import org.apache.twill.api.logging.LogEntry;
-import org.apache.twill.internal.json.JsonUtils;
-
-import java.lang.reflect.Type;
-
-/**
- * A {@link com.google.gson.Gson} decoder for {@link LogEntry}.
- */
-public final class LogEntryDecoder implements JsonDeserializer<LogEntry> {
-
-  @Override
-  public LogEntry deserialize(JsonElement json, Type typeOfT,
-                              JsonDeserializationContext context) throws JsonParseException {
-    if (!json.isJsonObject()) {
-      return null;
-    }
-    JsonObject jsonObj = json.getAsJsonObject();
-
-    final String name = JsonUtils.getAsString(jsonObj, "name");
-    final String host = JsonUtils.getAsString(jsonObj, "host");
-    final long timestamp = JsonUtils.getAsLong(jsonObj, "timestamp", 0);
-    LogEntry.Level l;
-    try {
-      l = LogEntry.Level.valueOf(JsonUtils.getAsString(jsonObj, "level"));
-    } catch (Exception e) {
-      l = LogEntry.Level.FATAL;
-    }
-    final LogEntry.Level logLevel = l;
-    final String className = JsonUtils.getAsString(jsonObj, "className");
-    final String method = JsonUtils.getAsString(jsonObj, "method");
-    final String file = JsonUtils.getAsString(jsonObj, "file");
-    final String line = JsonUtils.getAsString(jsonObj, "line");
-    final String thread = JsonUtils.getAsString(jsonObj, "thread");
-    final String message = JsonUtils.getAsString(jsonObj, "message");
-
-    final StackTraceElement[] stackTraces = context.deserialize(jsonObj.get("stackTraces").getAsJsonArray(),
-                                                                StackTraceElement[].class);
-
-    return new LogEntry() {
-      @Override
-      public String getLoggerName() {
-        return name;
-      }
-
-      @Override
-      public String getHost() {
-        return host;
-      }
-
-      @Override
-      public long getTimestamp() {
-        return timestamp;
-      }
-
-      @Override
-      public Level getLogLevel() {
-        return logLevel;
-      }
-
-      @Override
-      public String getSourceClassName() {
-        return className;
-      }
-
-      @Override
-      public String getSourceMethodName() {
-        return method;
-      }
-
-      @Override
-      public String getFileName() {
-        return file;
-      }
-
-      @Override
-      public int getLineNumber() {
-        if (line.equals("?")) {
-          return -1;
-        } else {
-          return Integer.parseInt(line);
-        }
-      }
-
-      @Override
-      public String getThreadName() {
-        return thread;
-      }
-
-      @Override
-      public String getMessage() {
-        return message;
-      }
-
-      @Override
-      public StackTraceElement[] getStackTraces() {
-        return stackTraces;
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e5c62e5c/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
index 30a5a41..4d45ad1 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
@@ -22,11 +22,15 @@ import org.apache.twill.api.TwillController;
 import org.apache.twill.api.TwillRunner;
 import org.apache.twill.api.logging.LogEntry;
 import org.apache.twill.api.logging.LogHandler;
+import org.apache.twill.api.logging.LogThrowable;
 import org.apache.twill.common.Services;
+import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
@@ -38,7 +42,8 @@ public class LogHandlerTestRun extends BaseYarnTest {
 
   @Test
   public void testLogHandler() throws ExecutionException, InterruptedException {
-    final CountDownLatch latch = new CountDownLatch(2);
+    final CountDownLatch latch = new CountDownLatch(3);
+    final Queue<LogThrowable> throwables = new ConcurrentLinkedQueue<LogThrowable>();
 
     LogHandler logHandler = new LogHandler() {
       @Override
@@ -48,6 +53,9 @@ public class LogHandlerTestRun extends BaseYarnTest {
           latch.countDown();
         } else if (logEntry.getMessage().equals("Running")) {
           latch.countDown();
+        } else if (logEntry.getMessage().equals("Got exception") && logEntry.getThrowable() != null) {
+          throwables.add(logEntry.getThrowable());
+          latch.countDown();
         }
       }
     };
@@ -59,6 +67,18 @@ public class LogHandlerTestRun extends BaseYarnTest {
 
     Services.getCompletionFuture(controller).get();
     latch.await(1, TimeUnit.SECONDS);
+
+    // Verify the log throwable
+    Assert.assertEquals(1, throwables.size());
+
+    LogThrowable t = throwables.poll();
+    Assert.assertEquals(RuntimeException.class.getName(), t.getClassName());
+    Assert.assertNotNull(t.getCause());
+    Assert.assertEquals(4, t.getStackTraces().length);
+
+    t = t.getCause();
+    Assert.assertEquals(Exception.class.getName(), t.getClassName());
+    Assert.assertEquals("Exception", t.getMessage());
   }
 
   /**
@@ -72,6 +92,16 @@ public class LogHandlerTestRun extends BaseYarnTest {
     @Override
     public void run() {
       LOG.info("Running");
+      try {
+        // Just throw some exception and log it
+        try {
+          throw new Exception("Exception");
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      } catch (Throwable t) {
+        LOG.error("Got exception", t);
+      }
     }
 
     @Override


[40/50] [abbrv] git commit: (TWILL-70) add debugging of containers

Posted by ch...@apache.org.
(TWILL-70) add debugging of containers


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

Branch: refs/heads/site
Commit: fe5cd78920e8ae95a90ecd88685f3c116f905051
Parents: 6149ea2
Author: anew <an...@continuuity.com>
Authored: Thu Apr 10 16:37:25 2014 -0700
Committer: anew <an...@continuuity.com>
Committed: Fri Apr 11 11:19:54 2014 -0700

----------------------------------------------------------------------
 .gitignore                                      |   8 +-
 .../org/apache/twill/api/TwillRunResources.java |   5 +
 .../internal/DefaultTwillRunResources.java      |  17 ++-
 .../twill/internal/ContainerLiveNodeData.java   |   8 +-
 .../org/apache/twill/internal/JvmOptions.java   |  85 +++++++++++++
 .../internal/TwillContainerController.java      |   5 +
 .../twill/internal/TwillContainerLauncher.java  |  88 ++++++++++---
 .../twill/internal/json/JvmOptionsCodec.java    | 114 +++++++++++++++++
 .../internal/json/TwillRunResourcesCodec.java   |   6 +-
 .../org/apache/twill/launcher/FindFreePort.java |  36 ++++++
 .../internal/json/JvmOptionsCodecTest.java      |  82 ++++++++++++
 .../appmaster/ApplicationMasterService.java     |  22 ++--
 .../internal/appmaster/RunningContainers.java   |  42 +++++-
 .../internal/appmaster/TrackerService.java      |  11 +-
 .../container/TwillContainerService.java        |  23 +++-
 .../apache/twill/yarn/YarnTwillPreparer.java    |  49 +++----
 .../twill/yarn/YarnTwillRunnerService.java      |  39 +++++-
 .../org/apache/twill/yarn/DebugTestRun.java     | 127 +++++++++++++++++++
 .../apache/twill/yarn/SessionExpireTestRun.java |   2 +-
 .../org/apache/twill/yarn/YarnTestSuite.java    |   3 +-
 20 files changed, 693 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 7aff00d..bcb6216 100644
--- a/.gitignore
+++ b/.gitignore
@@ -15,18 +15,14 @@ out/
 .DS_Store
 lib/
 .idea
-data/
 
 # Gradle Files & Dir #
 build/
-.gradle/
 .stickyStorage
 .build/
 target/
 
-# Node log
-npm-*.log
+# relics of test runs 
 logs/
+zookeeper.out
 
-# Singlenode run files. 
-data/

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-api/src/main/java/org/apache/twill/api/TwillRunResources.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/TwillRunResources.java b/twill-api/src/main/java/org/apache/twill/api/TwillRunResources.java
index 4c3d2e7..eea9619 100644
--- a/twill-api/src/main/java/org/apache/twill/api/TwillRunResources.java
+++ b/twill-api/src/main/java/org/apache/twill/api/TwillRunResources.java
@@ -48,4 +48,9 @@ public interface TwillRunResources {
    * @return id of the container the runnable is running in.
    */
   String getContainerId();
+
+  /**
+   * @return the debug port of the container's JVM, or null if not debug-enabled.
+   */
+  Integer getDebugPort();
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillRunResources.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillRunResources.java b/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillRunResources.java
index bd8f8f5..ae99daf 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillRunResources.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/DefaultTwillRunResources.java
@@ -28,14 +28,16 @@ public class DefaultTwillRunResources implements TwillRunResources {
   private final int virtualCores;
   private final int memoryMB;
   private final String host;
+  private final Integer debugPort;
 
   public DefaultTwillRunResources(int instanceId, String containerId,
-                                  int cores, int memoryMB, String host) {
+                                  int cores, int memoryMB, String host, Integer debugPort) {
     this.instanceId = instanceId;
     this.containerId = containerId;
     this.virtualCores = cores;
     this.memoryMB = memoryMB;
     this.host = host;
+    this.debugPort = debugPort;
   }
 
   /**
@@ -80,6 +82,11 @@ public class DefaultTwillRunResources implements TwillRunResources {
   }
 
   @Override
+  public Integer getDebugPort() {
+    return debugPort;
+  }
+
+  @Override
   public boolean equals(Object o) {
     if (!(o instanceof TwillRunResources)) {
       return false;
@@ -90,6 +97,7 @@ public class DefaultTwillRunResources implements TwillRunResources {
       host.equals(other.getHost()) &&
       (virtualCores == other.getVirtualCores()) &&
       (memoryMB == other.getMemoryMB());
+    // debugPort is ignored here
   }
 
   @Override
@@ -97,9 +105,10 @@ public class DefaultTwillRunResources implements TwillRunResources {
     int hash = 17;
     hash = 31 *  hash + containerId.hashCode();
     hash = 31 *  hash + host.hashCode();
-    hash = 31 *  hash + (int) (instanceId ^ (instanceId >>> 32));
-    hash = 31 *  hash + (int) (virtualCores ^ (virtualCores >>> 32));
-    hash = 31 *  hash + (int) (memoryMB ^ (memoryMB >>> 32));
+    hash = 31 *  hash + (instanceId ^ (instanceId >>> 32));
+    hash = 31 *  hash + (virtualCores ^ (virtualCores >>> 32));
+    hash = 31 *  hash + (memoryMB ^ (memoryMB >>> 32));
+    // debugPort is ignored here
     return hash;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-core/src/main/java/org/apache/twill/internal/ContainerLiveNodeData.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/ContainerLiveNodeData.java b/twill-core/src/main/java/org/apache/twill/internal/ContainerLiveNodeData.java
index 705943c..4c6f32e 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/ContainerLiveNodeData.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/ContainerLiveNodeData.java
@@ -24,10 +24,12 @@ public final class ContainerLiveNodeData {
 
   private final String containerId;
   private final String host;
+  private final String debugPort;
 
-  public ContainerLiveNodeData(String containerId, String host) {
+  public ContainerLiveNodeData(String containerId, String host, String debugPort) {
     this.containerId = containerId;
     this.host = host;
+    this.debugPort = debugPort;
   }
 
   public String getContainerId() {
@@ -37,4 +39,8 @@ public final class ContainerLiveNodeData {
   public String getHost() {
     return host;
   }
+
+  public String getDebugPort() {
+    return debugPort;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-core/src/main/java/org/apache/twill/internal/JvmOptions.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/JvmOptions.java b/twill-core/src/main/java/org/apache/twill/internal/JvmOptions.java
new file mode 100644
index 0000000..a260f5b
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/JvmOptions.java
@@ -0,0 +1,85 @@
+/*
+ * 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.twill.internal;
+
+import com.google.common.collect.ImmutableSet;
+
+import java.util.Set;
+
+/**
+ * A class that encapsulates the different options for starting Java in containers.
+ */
+public final class JvmOptions {
+
+  private final String extraOptions;
+  private final DebugOptions debugOptions;
+
+  public JvmOptions(String extraOptions, DebugOptions debugOptions) {
+    this.extraOptions = extraOptions;
+    this.debugOptions = debugOptions;
+  }
+
+  public String getExtraOptions() {
+    return extraOptions;
+  }
+
+  public DebugOptions getDebugOptions() {
+    return debugOptions;
+  }
+
+  /**
+   * Represents the debugging options of the JVM.
+   * <ul>
+   *   <li>whether debugging is enabled</li>
+   *   <li>whether VMs should suspend and wait for debugger</li>
+   *   <li>optionally specify what runnables should be debuggable (default is all)</li>
+   * </ul>
+   */
+  public static final class DebugOptions {
+    private final boolean doDebug;
+    private final boolean doSuspend;
+    private final Set<String> runnables;
+
+    public static final DebugOptions NO_DEBUG = new DebugOptions(false, false, null);
+
+    public DebugOptions(boolean doDebug, boolean doSuspend, Iterable<String> runnables) {
+      this.doDebug = doDebug;
+      this.doSuspend = doDebug && doSuspend;
+      this.runnables = doDebug && runnables != null ? ImmutableSet.copyOf(runnables) : null;
+    }
+
+    public boolean doDebug() {
+      return doDebug;
+    }
+
+    public boolean doSuspend() {
+      return doSuspend;
+    }
+
+    public Set<String> getRunnables() {
+      return runnables;
+    }
+
+    /**
+     * @return whether debugging is enabled for the given runnable name.
+     */
+    public boolean doDebug(String runnable) {
+      return doDebug && (runnables == null || runnables.contains(runnable));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
index bb46cd5..191e30c 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerController.java
@@ -33,4 +33,9 @@ public interface TwillContainerController extends ServiceController {
    * Any resources it hold will be releases and all pending futures will be cancelled.
    */
   void completed(int exitStatus);
+
+  /**
+   * @returns the container's live node data.
+   */
+  ContainerLiveNodeData getLiveNodeData();
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
index 430e63a..14bac7a 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
@@ -17,14 +17,19 @@
  */
 package org.apache.twill.internal;
 
+import com.google.common.base.Charsets;
+import com.google.common.collect.ImmutableList;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import com.google.gson.Gson;
+import com.google.gson.JsonElement;
 import org.apache.twill.api.LocalFile;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.RuntimeSpecification;
 import org.apache.twill.filesystem.Location;
 import org.apache.twill.internal.state.Message;
 import org.apache.twill.internal.state.StateNode;
+import org.apache.twill.launcher.FindFreePort;
 import org.apache.twill.launcher.TwillLauncher;
 import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.ZKClient;
@@ -34,6 +39,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.List;
 
 /**
  * This class helps launching a container.
@@ -48,12 +54,12 @@ public final class TwillContainerLauncher {
   private final ProcessLauncher.PrepareLaunchContext launchContext;
   private final ZKClient zkClient;
   private final int instanceCount;
-  private final String jvmOpts;
+  private final JvmOptions jvmOpts;
   private final int reservedMemory;
   private final Location secureStoreLocation;
 
   public TwillContainerLauncher(RuntimeSpecification runtimeSpec, ProcessLauncher.PrepareLaunchContext launchContext,
-                                ZKClient zkClient, int instanceCount, String jvmOpts, int reservedMemory,
+                                ZKClient zkClient, int instanceCount, JvmOptions jvmOpts, int reservedMemory,
                                 Location secureStoreLocation) {
     this.runtimeSpec = runtimeSpec;
     this.launchContext = launchContext;
@@ -110,24 +116,50 @@ public final class TwillContainerLauncher {
     }
 
     // Currently no reporting is supported for runnable containers
-    ProcessController<Void> processController = afterResources
+    ProcessLauncher.PrepareLaunchContext.MoreEnvironment afterEnvironment = afterResources
       .withEnvironment()
       .add(EnvKeys.TWILL_RUN_ID, runId.getId())
       .add(EnvKeys.TWILL_RUNNABLE_NAME, runtimeSpec.getName())
       .add(EnvKeys.TWILL_INSTANCE_ID, Integer.toString(instanceId))
-      .add(EnvKeys.TWILL_INSTANCE_COUNT, Integer.toString(instanceCount))
-      .withCommands()
-      .add("java",
-           "-Djava.io.tmpdir=tmp",
-           "-Dyarn.container=$" + EnvKeys.YARN_CONTAINER_ID,
-           "-Dtwill.runnable=$" + EnvKeys.TWILL_APP_NAME + ".$" + EnvKeys.TWILL_RUNNABLE_NAME,
-           "-cp", Constants.Files.LAUNCHER_JAR + ":" + classPath,
-           "-Xmx" + memory + "m",
-           jvmOpts,
-           TwillLauncher.class.getName(),
-           Constants.Files.CONTAINER_JAR,
-           mainClass.getName(),
-           Boolean.TRUE.toString())
+      .add(EnvKeys.TWILL_INSTANCE_COUNT, Integer.toString(instanceCount));
+
+    // assemble the command based on jvm options
+    ImmutableList.Builder<String> commandBuilder = ImmutableList.builder();
+    String firstCommand;
+    if (jvmOpts.getDebugOptions().doDebug(runtimeSpec.getName())) {
+      // for debugging we run a quick Java program to find a free port, then pass that port as the debug port and also
+      // as a System property to the runnable (Java has no general way to determine the port from within the JVM).
+      // PORT=$(java FindFreePort) && java -agentlib:jdwp=...,address=\$PORT -Dtwill.debug.port=\$PORT... TwillLauncher
+      // The $ must be escaped, otherwise it gets expanded (to "") before the command is submitted.
+      String suspend = jvmOpts.getDebugOptions().doSuspend() ? "y" : "n";
+      firstCommand = "TWILL_DEBUG_PORT=$($JAVA_HOME/bin/java";
+      commandBuilder.add("-cp", Constants.Files.LAUNCHER_JAR,
+                         FindFreePort.class.getName() + ")",
+                         "&&", // this will stop if FindFreePort fails
+                         "$JAVA_HOME/bin/java",
+                         "-agentlib:jdwp=transport=dt_socket,server=y,suspend=" + suspend + "," +
+                           "address=\\$TWILL_DEBUG_PORT",
+                         "-Dtwill.debug.port=\\$TWILL_DEBUG_PORT"
+                         );
+    } else {
+      firstCommand = "$JAVA_HOME/bin/java";
+    }
+    commandBuilder.add("-Djava.io.tmpdir=tmp",
+                       "-Dyarn.container=$" + EnvKeys.YARN_CONTAINER_ID,
+                       "-Dtwill.runnable=$" + EnvKeys.TWILL_APP_NAME + ".$" + EnvKeys.TWILL_RUNNABLE_NAME,
+                       "-cp", Constants.Files.LAUNCHER_JAR + ":" + classPath,
+                       "-Xmx" + memory + "m");
+    if (jvmOpts.getExtraOptions() != null) {
+      commandBuilder.add(jvmOpts.getExtraOptions());
+    }
+    commandBuilder.add(TwillLauncher.class.getName(),
+                       Constants.Files.CONTAINER_JAR,
+                       mainClass.getName(),
+                       Boolean.TRUE.toString());
+    List<String> command = commandBuilder.build();
+
+    ProcessController<Void> processController = afterEnvironment
+      .withCommands().add(firstCommand, command.toArray(new String[command.size()]))
       .redirectOutput(Constants.STDOUT).redirectError(Constants.STDERR)
       .launch();
 
@@ -140,6 +172,7 @@ public final class TwillContainerLauncher {
                                                           implements TwillContainerController {
 
     private final ProcessController<Void> processController;
+    private volatile ContainerLiveNodeData liveData;
 
     protected TwillContainerControllerImpl(ZKClient zkClient, RunId runId,
                                            ProcessController<Void> processController) {
@@ -159,7 +192,23 @@ public final class TwillContainerLauncher {
 
     @Override
     protected void instanceNodeUpdated(NodeData nodeData) {
-      // No-op
+      if (nodeData == null ||  nodeData.getData() == null) {
+        LOG.warn("Instance node was updated but data is null.");
+        return;
+      }
+      try {
+        Gson gson = new Gson();
+        JsonElement json = gson.fromJson(new String(nodeData.getData(), Charsets.UTF_8), JsonElement.class);
+        if (json.isJsonObject()) {
+          JsonElement data = json.getAsJsonObject().get("data");
+          if (data != null) {
+            this.liveData = gson.fromJson(data, ContainerLiveNodeData.class);
+            LOG.info("Container LiveNodeData updated: " + new String(nodeData.getData(), Charsets.UTF_8));
+          }
+        }
+      } catch (Throwable t) {
+        LOG.warn("Error deserializing updated instance node data", t);
+      }
     }
 
     @Override
@@ -186,6 +235,11 @@ public final class TwillContainerLauncher {
     }
 
     @Override
+    public ContainerLiveNodeData getLiveNodeData() {
+      return liveData;
+    }
+
+    @Override
     public void kill() {
       processController.cancel();
     }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-core/src/main/java/org/apache/twill/internal/json/JvmOptionsCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/JvmOptionsCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/JvmOptionsCodec.java
new file mode 100644
index 0000000..a1b7eb9
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/JvmOptionsCodec.java
@@ -0,0 +1,114 @@
+/*
+ * 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.twill.internal.json;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.io.InputSupplier;
+import com.google.common.io.OutputSupplier;
+import com.google.common.reflect.TypeToken;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParseException;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
+import org.apache.twill.internal.JvmOptions;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.io.Writer;
+import java.lang.reflect.Type;
+import java.util.Set;
+
+/**
+ * Gson codec for {@link JvmOptions}.
+ */
+public class JvmOptionsCodec implements JsonSerializer<JvmOptions>, JsonDeserializer<JvmOptions> {
+
+  private static final Gson GSON = new GsonBuilder().registerTypeAdapter(JvmOptions.class, new JvmOptionsCodec())
+                                                    .registerTypeAdapter(JvmOptions.DebugOptions.class,
+                                                                         new DebugOptionsCodec())
+                                                    .create();
+
+  public static void encode(JvmOptions jvmOptions, OutputSupplier<? extends Writer> writerSupplier) throws IOException {
+    Writer writer = writerSupplier.getOutput();
+    try {
+      GSON.toJson(jvmOptions, writer);
+    } finally {
+      writer.close();
+    }
+  }
+
+  public static JvmOptions decode(InputSupplier<? extends Reader> readerSupplier) throws IOException {
+    Reader reader = readerSupplier.getInput();
+    try {
+      return GSON.fromJson(reader, JvmOptions.class);
+    } finally {
+      reader.close();
+    }
+  }
+
+  @Override
+  public JvmOptions deserialize(JsonElement json, Type type, JsonDeserializationContext context)
+    throws JsonParseException {
+    JsonObject jsonObj = json.getAsJsonObject();
+    String extraOptions = context.deserialize(jsonObj.get("extraOptions"), String.class);
+    JvmOptions.DebugOptions debugOptions = context.deserialize(jsonObj.get("debugOptions"),
+                                                               JvmOptions.DebugOptions.class);
+    return new JvmOptions(extraOptions, debugOptions);
+  }
+
+  @Override
+  public JsonElement serialize(JvmOptions jvmOptions, Type type, JsonSerializationContext context) {
+    JsonObject json = new JsonObject();
+    json.add("extraOptions", context.serialize(jvmOptions.getExtraOptions()));
+    json.add("debugOptions", context.serialize(jvmOptions.getDebugOptions()));
+    return json;
+  }
+
+  private static class DebugOptionsCodec
+    implements JsonSerializer<JvmOptions.DebugOptions>, JsonDeserializer<JvmOptions.DebugOptions> {
+
+    @Override
+    public JvmOptions.DebugOptions deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context)
+      throws JsonParseException {
+      JsonObject jsonObj = json.getAsJsonObject();
+      Boolean doDebug = context.deserialize(jsonObj.get("doDebug"), Boolean.class);
+      if (!doDebug) {
+        return JvmOptions.DebugOptions.NO_DEBUG;
+      }
+      Boolean doSuspend = context.deserialize(jsonObj.get("doSuspend"), Boolean.class);
+      Set<String> runnables = context.deserialize(jsonObj.get("runnables"), new TypeToken<Set<String>>() { }.getType());
+      return new JvmOptions.DebugOptions(true, doSuspend, ImmutableSet.copyOf(runnables));
+    }
+
+    @Override
+    public JsonElement serialize(JvmOptions.DebugOptions src, Type typeOfSrc, JsonSerializationContext context) {
+      JsonObject json = new JsonObject();
+      json.add("doDebug", context.serialize(src.doDebug()));
+      json.add("doSuspend", context.serialize(src.doSuspend()));
+      json.add("runnables", context.serialize(src.getRunnables()));
+      return json;
+    }
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunResourcesCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunResourcesCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunResourcesCodec.java
index c39fa16..552469b 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunResourcesCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/TwillRunResourcesCodec.java
@@ -44,6 +44,9 @@ public final class TwillRunResourcesCodec implements JsonSerializer<TwillRunReso
     json.addProperty("host", src.getHost());
     json.addProperty("memoryMB", src.getMemoryMB());
     json.addProperty("virtualCores", src.getVirtualCores());
+    if (src.getDebugPort() != null) {
+      json.addProperty("debugPort", src.getDebugPort());
+    }
 
     return json;
   }
@@ -56,6 +59,7 @@ public final class TwillRunResourcesCodec implements JsonSerializer<TwillRunReso
                                         jsonObj.get("containerId").getAsString(),
                                         jsonObj.get("virtualCores").getAsInt(),
                                         jsonObj.get("memoryMB").getAsInt(),
-                                        jsonObj.get("host").getAsString());
+                                        jsonObj.get("host").getAsString(),
+                                        jsonObj.has("debugPort") ? jsonObj.get("debugPort").getAsInt() : null);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-core/src/main/java/org/apache/twill/launcher/FindFreePort.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/launcher/FindFreePort.java b/twill-core/src/main/java/org/apache/twill/launcher/FindFreePort.java
new file mode 100644
index 0000000..89e6b57
--- /dev/null
+++ b/twill-core/src/main/java/org/apache/twill/launcher/FindFreePort.java
@@ -0,0 +1,36 @@
+/*
+ * 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.twill.launcher;
+
+import java.net.ServerSocket;
+
+/**
+ * Utility main class to find a free port on a machine.
+ */
+public class FindFreePort {
+
+  public static void main(String[] args) throws Exception {
+    ServerSocket socket = new ServerSocket(0);
+    try {
+      System.out.println(socket.getLocalPort());
+    } finally {
+      socket.close();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-core/src/test/java/org/apache/twill/internal/json/JvmOptionsCodecTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/json/JvmOptionsCodecTest.java b/twill-core/src/test/java/org/apache/twill/internal/json/JvmOptionsCodecTest.java
new file mode 100644
index 0000000..66187f2
--- /dev/null
+++ b/twill-core/src/test/java/org/apache/twill/internal/json/JvmOptionsCodecTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.twill.internal.json;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.io.InputSupplier;
+import com.google.common.io.OutputSupplier;
+import org.apache.twill.internal.JvmOptions;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.io.Writer;
+
+/**
+ * Tests the JvmOptions Codec.
+ */
+public class JvmOptionsCodecTest {
+
+  @Test
+  public void testNoNulls() throws Exception {
+    JvmOptions options = new JvmOptions("-version",
+                                        new JvmOptions.DebugOptions(true, false, ImmutableSet.of("one", "two")));
+    final StringWriter writer = new StringWriter();
+    JvmOptionsCodec.encode(options, new OutputSupplier<Writer>() {
+      @Override
+      public Writer getOutput() throws IOException {
+        return writer;
+      }
+    });
+    JvmOptions options1 = JvmOptionsCodec.decode(new InputSupplier<Reader>() {
+      @Override
+      public Reader getInput() throws IOException {
+        return new StringReader(writer.toString());
+      }
+    });
+    Assert.assertEquals(options.getExtraOptions(), options1.getExtraOptions());
+    Assert.assertEquals(options.getDebugOptions().doDebug(), options1.getDebugOptions().doDebug());
+    Assert.assertEquals(options.getDebugOptions().doSuspend(), options1.getDebugOptions().doSuspend());
+    Assert.assertEquals(options.getDebugOptions().getRunnables(), options1.getDebugOptions().getRunnables());
+  }
+
+  @Test
+  public void testSomeNulls() throws Exception {
+    JvmOptions options = new JvmOptions(null, new JvmOptions.DebugOptions(false, false, null));
+    final StringWriter writer = new StringWriter();
+    JvmOptionsCodec.encode(options, new OutputSupplier<Writer>() {
+      @Override
+      public Writer getOutput() throws IOException {
+        return writer;
+      }
+    });
+    JvmOptions options1 = JvmOptionsCodec.decode(new InputSupplier<Reader>() {
+      @Override
+      public Reader getInput() throws IOException {
+        return new StringReader(writer.toString());
+      }
+    });
+    Assert.assertEquals(options.getExtraOptions(), options1.getExtraOptions());
+    Assert.assertEquals(options.getDebugOptions().doDebug(), options1.getDebugOptions().doDebug());
+    Assert.assertEquals(options.getDebugOptions().doSuspend(), options1.getDebugOptions().doSuspend());
+    Assert.assertEquals(options.getDebugOptions().getRunnables(), options1.getDebugOptions().getRunnables());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
index 3a6ce20..f1ad20e 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
@@ -31,7 +31,6 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multiset;
 import com.google.common.collect.Sets;
-import com.google.common.io.CharStreams;
 import com.google.common.io.Files;
 import com.google.common.io.InputSupplier;
 import com.google.common.reflect.TypeToken;
@@ -54,6 +53,7 @@ import org.apache.twill.api.Command;
 import org.apache.twill.api.EventHandler;
 import org.apache.twill.api.EventHandlerSpecification;
 import org.apache.twill.api.LocalFile;
+import org.apache.twill.api.ResourceReport;
 import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.RuntimeSpecification;
@@ -66,9 +66,11 @@ import org.apache.twill.internal.Configs;
 import org.apache.twill.internal.Constants;
 import org.apache.twill.internal.DefaultTwillRunResources;
 import org.apache.twill.internal.EnvKeys;
+import org.apache.twill.internal.JvmOptions;
 import org.apache.twill.internal.ProcessLauncher;
 import org.apache.twill.internal.TwillContainerLauncher;
 import org.apache.twill.internal.ZKServiceDecorator;
+import org.apache.twill.internal.json.JvmOptionsCodec;
 import org.apache.twill.internal.json.LocalFileCodec;
 import org.apache.twill.internal.json.TwillSpecificationAdapter;
 import org.apache.twill.internal.kafka.EmbeddedKafkaServer;
@@ -124,7 +126,7 @@ public final class ApplicationMasterService extends AbstractTwillService {
   private final ExpectedContainers expectedContainers;
   private final TrackerService trackerService;
   private final YarnAMClient amClient;
-  private final String jvmOpts;
+  private final JvmOptions jvmOpts;
   private final int reservedMemory;
   private final EventHandler eventHandler;
   private final Location applicationLocation;
@@ -159,17 +161,21 @@ public final class ApplicationMasterService extends AbstractTwillService {
     });
     expectedContainers = initExpectedContainers(twillSpec);
     runningContainers = initRunningContainers(amClient.getContainerId(), amClient.getHost());
-    trackerService = new TrackerService(runningContainers.getResourceReport(), amClient.getHost());
+    trackerService = new TrackerService(new Supplier<ResourceReport>() {
+      @Override
+      public ResourceReport get() {
+        return runningContainers.getResourceReport();
+      }
+    }, amClient.getHost());
     eventHandler = createEventHandler(twillSpec);
   }
 
-  private String loadJvmOptions() throws IOException {
+  private JvmOptions loadJvmOptions() throws IOException {
     final File jvmOptsFile = new File(Constants.Files.JVM_OPTIONS);
     if (!jvmOptsFile.exists()) {
-      return "";
+      return new JvmOptions(null, JvmOptions.DebugOptions.NO_DEBUG);
     }
-
-    return CharStreams.toString(new InputSupplier<Reader>() {
+    return JvmOptionsCodec.decode(new InputSupplier<Reader>() {
       @Override
       public Reader getInput() throws IOException {
         return new FileReader(jvmOptsFile);
@@ -220,7 +226,7 @@ public final class ApplicationMasterService extends AbstractTwillService {
       appMasterContainerId.toString(),
       Integer.parseInt(System.getenv(EnvKeys.YARN_CONTAINER_VIRTUAL_CORES)),
       Integer.parseInt(System.getenv(EnvKeys.YARN_CONTAINER_MEMORY_MB)),
-      appMasterHost);
+      appMasterHost, null);
     String appId = appMasterContainerId.getApplicationAttemptId().getApplicationId().toString();
     return new RunningContainers(appId, appMasterResources);
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
index 63e3db8..881acc9 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
@@ -36,6 +36,7 @@ import org.apache.twill.api.ServiceController;
 import org.apache.twill.api.TwillRunResources;
 import org.apache.twill.internal.ContainerExitCodes;
 import org.apache.twill.internal.ContainerInfo;
+import org.apache.twill.internal.ContainerLiveNodeData;
 import org.apache.twill.internal.DefaultResourceReport;
 import org.apache.twill.internal.DefaultTwillRunResources;
 import org.apache.twill.internal.RunIds;
@@ -115,12 +116,12 @@ final class RunningContainers {
       TwillContainerController controller = launcher.start(runId, instanceId,
                                                            TwillContainerMain.class, "$HADOOP_CONF_DIR");
       containers.put(runnableName, containerInfo.getId(), controller);
-
-      TwillRunResources resources = new DefaultTwillRunResources(instanceId,
+      TwillRunResources resources = new DynamicTwillRunResources(instanceId,
                                                                  containerInfo.getId(),
                                                                  containerInfo.getVirtualCores(),
                                                                  containerInfo.getMemoryMB(),
-                                                                 containerInfo.getHost().getHostName());
+                                                                 containerInfo.getHost().getHostName(),
+                                                                 controller);
       resourceReport.addRunResources(runnableName, resources);
 
       if (startSequence.isEmpty() || !runnableName.equals(startSequence.peekLast())) {
@@ -290,7 +291,8 @@ final class RunningContainers {
 
   /**
    * Handle completion of container.
-   * @param status The completion status.
+   *
+   * @param status           The completion status.
    * @param restartRunnables Set of runnable names that requires restart.
    */
   void handleCompleted(YarnContainerStatus status, Multiset<String> restartRunnables) {
@@ -434,4 +436,36 @@ final class RunningContainers {
     String id = runId.getId();
     return Integer.parseInt(id.substring(id.lastIndexOf('-') + 1));
   }
+
+  /**
+   * A helper class that overrides the debug port of the resources with the live info from the container controller.
+   */
+  private static class DynamicTwillRunResources extends DefaultTwillRunResources {
+
+    private final TwillContainerController controller;
+    private Integer dynamicDebugPort = null;
+
+    private DynamicTwillRunResources(int instanceId, String containerId,
+                                     int cores, int memoryMB, String host,
+                                     TwillContainerController controller) {
+      super(instanceId, containerId, cores, memoryMB, host, null);
+      this.controller = controller;
+    }
+
+    @Override
+    public synchronized Integer getDebugPort() {
+      if (dynamicDebugPort == null) {
+        ContainerLiveNodeData liveData = controller.getLiveNodeData();
+        if (liveData != null && liveData.getDebugPort() != null) {
+          try {
+            dynamicDebugPort = Integer.parseInt(liveData.getDebugPort());
+          } catch (NumberFormatException e) {
+            LOG.warn("Live data for {} has debug port of '{}' which cannot be parsed as a number",
+                     getContainerId(), liveData.getDebugPort());
+          }
+        }
+      }
+      return dynamicDebugPort;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
index a9553c9..6165c6f 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/TrackerService.java
@@ -17,6 +17,7 @@
  */
 package org.apache.twill.internal.appmaster;
 
+import com.google.common.base.Supplier;
 import com.google.common.util.concurrent.AbstractIdleService;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.twill.api.ResourceReport;
@@ -84,7 +85,7 @@ public final class TrackerService extends AbstractIdleService {
   private InetSocketAddress bindAddress;
   private URL url;
   private final ChannelGroup channelGroup;
-  private final ResourceReport resourceReport;
+  private final Supplier<ResourceReport> resourceReport;
 
   /**
    * Initialize the service.
@@ -92,7 +93,7 @@ public final class TrackerService extends AbstractIdleService {
    * @param resourceReport live report that the service will return to clients.
    * @param appMasterHost the application master host.
    */
-  public TrackerService(ResourceReport resourceReport, String appMasterHost) {
+  public TrackerService(Supplier<ResourceReport> resourceReport, String appMasterHost) {
     this.channelGroup = new DefaultChannelGroup("appMasterTracker");
     this.resourceReport = resourceReport;
     this.host = appMasterHost;
@@ -165,10 +166,10 @@ public final class TrackerService extends AbstractIdleService {
    * the host and port set when this application master registered itself to the resource manager.
    */
   public class ReportHandler extends SimpleChannelUpstreamHandler {
-    private final ResourceReport report;
+    private final Supplier<ResourceReport> report;
     private final ResourceReportAdapter reportAdapter;
 
-    public ReportHandler(ResourceReport report) {
+    public ReportHandler(Supplier<ResourceReport> report) {
       this.report = report;
       this.reportAdapter = ResourceReportAdapter.create();
     }
@@ -201,7 +202,7 @@ public final class TrackerService extends AbstractIdleService {
 
       ChannelBuffer content = ChannelBuffers.dynamicBuffer();
       Writer writer = new OutputStreamWriter(new ChannelBufferOutputStream(content), CharsetUtil.UTF_8);
-      reportAdapter.toJson(report, writer);
+      reportAdapter.toJson(report.get(), writer);
       try {
         writer.close();
       } catch (IOException e1) {

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
index 9890f17..b212a29 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/container/TwillContainerService.java
@@ -57,7 +57,6 @@ public final class TwillContainerService extends AbstractTwillService {
 
   private final TwillRunnableSpecification specification;
   private final ClassLoader classLoader;
-  private final ContainerLiveNodeData containerLiveNode;
   private final BasicTwillContext context;
   private final ZKServiceDecorator serviceDelegate;
   private ExecutorService commandExecutor;
@@ -70,10 +69,10 @@ public final class TwillContainerService extends AbstractTwillService {
 
     this.specification = specification;
     this.classLoader = classLoader;
-    this.serviceDelegate = new ZKServiceDecorator(zkClient, runId, createLiveNodeSupplier(), new ServiceDelegate());
+    this.serviceDelegate = new ZKServiceDecorator(zkClient, runId,
+                                                  createLiveNodeSupplier(createLiveNodeData(containerInfo)),
+                                                  new ServiceDelegate());
     this.context = context;
-    this.containerLiveNode = new ContainerLiveNodeData(containerInfo.getId(),
-                                                       containerInfo.getHost().getCanonicalHostName());
   }
 
   private ListenableFuture<String> processMessage(final String messageId, final Message message) {
@@ -105,11 +104,22 @@ public final class TwillContainerService extends AbstractTwillService {
     return result;
   }
 
-  private Supplier<? extends JsonElement> createLiveNodeSupplier() {
+  private ContainerLiveNodeData createLiveNodeData(ContainerInfo containerInfo) {
+    // if debugging is enabled, log the port and register it in service discovery.
+    String debugPort = System.getProperty("twill.debug.port");
+    if (debugPort != null) {
+      LOG.info("JVM is listening for debugger on port {}", debugPort);
+    }
+    return new ContainerLiveNodeData(containerInfo.getId(),
+                                     containerInfo.getHost().getCanonicalHostName(),
+                                     debugPort);
+  }
+
+  private Supplier<? extends JsonElement> createLiveNodeSupplier(final ContainerLiveNodeData data) {
     return new Supplier<JsonElement>() {
       @Override
       public JsonElement get() {
-        return new Gson().toJsonTree(containerLiveNode);
+        return new Gson().toJsonTree(data);
       }
     };
   }
@@ -123,6 +133,7 @@ public final class TwillContainerService extends AbstractTwillService {
 
     @Override
     protected void startUp() throws Exception {
+
       commandExecutor = Executors.newSingleThreadExecutor(
         Threads.createDaemonThreadFactory("runnable-command-executor"));
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
index 8c96629..a02efea 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
@@ -35,7 +35,6 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
 import com.google.common.io.ByteStreams;
-import com.google.common.io.CharStreams;
 import com.google.common.io.OutputSupplier;
 import com.google.common.reflect.TypeToken;
 import com.google.gson.GsonBuilder;
@@ -63,6 +62,7 @@ import org.apache.twill.internal.DefaultLocalFile;
 import org.apache.twill.internal.DefaultRuntimeSpecification;
 import org.apache.twill.internal.DefaultTwillSpecification;
 import org.apache.twill.internal.EnvKeys;
+import org.apache.twill.internal.JvmOptions;
 import org.apache.twill.internal.LogOnlyEventHandler;
 import org.apache.twill.internal.ProcessController;
 import org.apache.twill.internal.ProcessLauncher;
@@ -70,6 +70,7 @@ import org.apache.twill.internal.RunIds;
 import org.apache.twill.internal.appmaster.ApplicationMasterMain;
 import org.apache.twill.internal.container.TwillContainerMain;
 import org.apache.twill.internal.json.ArgumentsCodec;
+import org.apache.twill.internal.json.JvmOptionsCodec;
 import org.apache.twill.internal.json.LocalFileCodec;
 import org.apache.twill.internal.json.TwillSpecificationAdapter;
 import org.apache.twill.internal.utils.Dependencies;
@@ -77,6 +78,7 @@ import org.apache.twill.internal.utils.Paths;
 import org.apache.twill.internal.yarn.YarnAppClient;
 import org.apache.twill.internal.yarn.YarnApplicationReport;
 import org.apache.twill.internal.yarn.YarnUtils;
+import org.apache.twill.launcher.FindFreePort;
 import org.apache.twill.launcher.TwillLauncher;
 import org.apache.twill.zookeeper.ZKClient;
 import org.apache.twill.zookeeper.ZKClients;
@@ -105,14 +107,13 @@ import java.util.jar.JarOutputStream;
 final class YarnTwillPreparer implements TwillPreparer {
 
   private static final Logger LOG = LoggerFactory.getLogger(YarnTwillPreparer.class);
-  private static final String KAFKA_ARCHIVE = "kafka-0.7.2.tgz";
 
   private final YarnConfiguration yarnConfig;
   private final TwillSpecification twillSpec;
   private final YarnAppClient yarnAppClient;
   private final ZKClient zkClient;
   private final LocationFactory locationFactory;
-  private final Supplier<String> jvmOpts;
+  private final JvmOptions jvmOpts;
   private final YarnTwillControllerFactory controllerFactory;
   private final RunId runId;
 
@@ -127,14 +128,14 @@ final class YarnTwillPreparer implements TwillPreparer {
   private String user;
 
   YarnTwillPreparer(YarnConfiguration yarnConfig, TwillSpecification twillSpec, YarnAppClient yarnAppClient,
-                    ZKClient zkClient, LocationFactory locationFactory, Supplier<String> jvmOpts,
+                    ZKClient zkClient, LocationFactory locationFactory, Supplier<JvmOptions> jvmOpts,
                     YarnTwillControllerFactory controllerFactory) {
     this.yarnConfig = yarnConfig;
     this.twillSpec = twillSpec;
     this.yarnAppClient = yarnAppClient;
     this.zkClient = ZKClients.namespace(zkClient, "/" + twillSpec.getName());
     this.locationFactory = locationFactory;
-    this.jvmOpts = jvmOpts;
+    this.jvmOpts = jvmOpts.get();
     this.controllerFactory = controllerFactory;
     this.runId = RunIds.generate();
     this.credentials = createCredentials();
@@ -235,15 +236,13 @@ final class YarnTwillPreparer implements TwillPreparer {
           // Local files declared by runnables
           Multimap<String, LocalFile> runnableLocalFiles = HashMultimap.create();
 
-          String vmOpts = jvmOpts.get();
-
           createAppMasterJar(createBundler(), localFiles);
           createContainerJar(createBundler(), localFiles);
           populateRunnableLocalFiles(twillSpec, runnableLocalFiles);
           saveSpecification(twillSpec, runnableLocalFiles, localFiles);
           saveLogback(localFiles);
           saveLauncher(localFiles);
-          saveVmOptions(vmOpts, localFiles);
+          saveJvmOptions(jvmOpts, localFiles);
           saveArguments(new Arguments(arguments, runnableArgs), localFiles);
           saveLocalFiles(localFiles, ImmutableSet.of(Constants.Files.TWILL_SPEC,
                                                      Constants.Files.LOGBACK_TEMPLATE,
@@ -259,23 +258,24 @@ final class YarnTwillPreparer implements TwillPreparer {
           //     false
           return launcher.prepareLaunch(
             ImmutableMap.<String, String>builder()
-              .put(EnvKeys.TWILL_FS_USER, fsUser)
-              .put(EnvKeys.TWILL_APP_DIR, getAppLocation().toURI().toASCIIString())
-              .put(EnvKeys.TWILL_ZK_CONNECT, zkClient.getConnectString())
-              .put(EnvKeys.TWILL_RUN_ID, runId.getId())
-              .put(EnvKeys.TWILL_RESERVED_MEMORY_MB, Integer.toString(reservedMemory))
-              .put(EnvKeys.TWILL_APP_NAME, twillSpec.getName()).build(),
-            localFiles.values(), credentials)
+                        .put(EnvKeys.TWILL_FS_USER, fsUser)
+                        .put(EnvKeys.TWILL_APP_DIR, getAppLocation().toURI().toASCIIString())
+                        .put(EnvKeys.TWILL_ZK_CONNECT, zkClient.getConnectString())
+                        .put(EnvKeys.TWILL_RUN_ID, runId.getId())
+                        .put(EnvKeys.TWILL_RESERVED_MEMORY_MB, Integer.toString(reservedMemory))
+                        .put(EnvKeys.TWILL_APP_NAME, twillSpec.getName()).build(),
+            localFiles.values(), credentials
+          )
             .noResources()
             .noEnvironment()
             .withCommands().add(
-              "java",
+              "$JAVA_HOME/bin/java",
               "-Djava.io.tmpdir=tmp",
               "-Dyarn.appId=$" + EnvKeys.YARN_APP_ID_STR,
               "-Dtwill.app=$" + EnvKeys.TWILL_APP_NAME,
               "-cp", Constants.Files.LAUNCHER_JAR + ":$HADOOP_CONF_DIR",
               "-Xmx" + (Constants.APP_MASTER_MEMORY_MB - Constants.APP_MASTER_RESERVED_MEMORY_MB) + "m",
-              vmOpts,
+              jvmOpts.getExtraOptions() == null ? "" : jvmOpts.getExtraOptions(),
               TwillLauncher.class.getName(),
               Constants.Files.APP_MASTER_JAR,
               ApplicationMasterMain.class.getName(),
@@ -457,8 +457,10 @@ final class YarnTwillPreparer implements TwillPreparer {
     Location location = createTempLocation(Constants.Files.LAUNCHER_JAR);
 
     final String launcherName = TwillLauncher.class.getName();
+    final String portFinderName = FindFreePort.class.getName();
 
     // Create a jar file with the TwillLauncher optionally a json serialized classpath.json in it.
+    // Also a little utility to find a free port, used for debugging.
     final JarOutputStream jarOut = new JarOutputStream(location.getOutputStream());
     ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
     if (classLoader == null) {
@@ -467,7 +469,7 @@ final class YarnTwillPreparer implements TwillPreparer {
     Dependencies.findClassDependencies(classLoader, new Dependencies.ClassAcceptor() {
       @Override
       public boolean accept(String className, URL classUrl, URL classPathUrl) {
-        Preconditions.checkArgument(className.startsWith(launcherName),
+        Preconditions.checkArgument(className.startsWith(launcherName) || className.equals(portFinderName),
                                     "Launcher jar should not have dependencies: %s", className);
         try {
           jarOut.putNextEntry(new JarEntry(className.replace('.', '/') + ".class"));
@@ -482,7 +484,7 @@ final class YarnTwillPreparer implements TwillPreparer {
         }
         return true;
       }
-    }, TwillLauncher.class.getName());
+    }, launcherName, portFinderName);
 
     try {
       if (!classPaths.isEmpty()) {
@@ -497,14 +499,15 @@ final class YarnTwillPreparer implements TwillPreparer {
     localFiles.put(Constants.Files.LAUNCHER_JAR, createLocalFile(Constants.Files.LAUNCHER_JAR, location));
   }
 
-  private void saveVmOptions(String opts, Map<String, LocalFile> localFiles) throws IOException {
-    if (opts.isEmpty()) {
+  private void saveJvmOptions(JvmOptions opts, Map<String, LocalFile> localFiles) throws IOException {
+    if ((opts.getExtraOptions() == null || opts.getExtraOptions().isEmpty()) &&
+      JvmOptions.DebugOptions.NO_DEBUG.equals(opts.getDebugOptions())) {
       // If no vm options, no need to localize the file.
       return;
     }
-    LOG.debug("Copy {}", Constants.Files.JVM_OPTIONS);
+    LOG.debug("Create and copy {}", Constants.Files.JVM_OPTIONS);
     final Location location = createTempLocation(Constants.Files.JVM_OPTIONS);
-    CharStreams.write(opts, new OutputSupplier<Writer>() {
+    JvmOptionsCodec.encode(opts, new OutputSupplier<Writer>() {
       @Override
       public Writer getOutput() throws IOException {
         return new OutputStreamWriter(location.getOutputStream(), Charsets.UTF_8);

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
index 72ea58b..4ecfb0e 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
@@ -65,6 +65,7 @@ import org.apache.twill.filesystem.HDFSLocationFactory;
 import org.apache.twill.filesystem.Location;
 import org.apache.twill.filesystem.LocationFactory;
 import org.apache.twill.internal.Constants;
+import org.apache.twill.internal.JvmOptions;
 import org.apache.twill.internal.ProcessController;
 import org.apache.twill.internal.RunIds;
 import org.apache.twill.internal.SingleRunnableApplication;
@@ -132,7 +133,9 @@ public final class YarnTwillRunnerService extends AbstractIdleService implements
 
   private Iterable<LiveInfo> liveInfos;
   private Cancellable watchCancellable;
-  private volatile String jvmOptions = "";
+
+  private volatile String jvmOptions = null;
+  private volatile JvmOptions.DebugOptions debugOptions = JvmOptions.DebugOptions.NO_DEBUG;
 
   public YarnTwillRunnerService(YarnConfiguration config, String zkConnect) {
     this(config, zkConnect, new HDFSLocationFactory(getFileSystem(config), "/twill"));
@@ -169,6 +172,38 @@ public final class YarnTwillRunnerService extends AbstractIdleService implements
     return jvmOptions;
   }
 
+  /**
+   * Disable debugging for runnables. It only affects applications that are started after this
+   * method is called. Note that debugging is disabled by default, and this method is only needed
+   * to revert a previous call of @link #enableDebugging.
+   */
+  public void disableDebugging() {
+    this.debugOptions = JvmOptions.DebugOptions.NO_DEBUG;
+  }
+
+  /**
+   * Enable debugging for runnables, without suspending the virtual machine to wait for the debugger.
+   * This replaces any previous debug settings and only affects applications that are started after
+   * this method is called.
+   * @param runnables the names of runnables to enable for debugging.
+   */
+  public void enableDebugging(String ... runnables) {
+    this.debugOptions = new JvmOptions.DebugOptions(true, false, ImmutableSet.copyOf(runnables));
+  }
+
+  /**
+   * Enable debugging for runnables. This replaces any previous debug settings and only affects
+   * applications that are started after this method is called.
+   * @param doSuspend whether the virtual machines should be supended until the debugger connects. This
+   *                  option allows to debug a container from the very beginning. Note that in that case,
+   *                  the container cannot notify the controller of its debug port until the debugger is
+   *                  attached - you must figure out where it is running using the YARN console or APIs.
+   * @param runnables the names of runnables to enable for debugging.
+   */
+  public void enableDebugging(boolean doSuspend, String ... runnables) {
+    this.debugOptions = new JvmOptions.DebugOptions(true, false, ImmutableSet.copyOf(runnables));
+  }
+
   @Override
   public Cancellable scheduleSecureStoreUpdate(final SecureStoreUpdater updater,
                                                long initialDelay, long delay, TimeUnit unit) {
@@ -239,7 +274,7 @@ public final class YarnTwillRunnerService extends AbstractIdleService implements
     final String appName = twillSpec.getName();
 
     return new YarnTwillPreparer(yarnConfig, twillSpec, yarnAppClient, zkClientService, locationFactory,
-                                 Suppliers.ofInstance(jvmOptions),
+                                 Suppliers.ofInstance(new JvmOptions(jvmOptions, debugOptions)),
                                  new YarnTwillControllerFactory() {
       @Override
       public YarnTwillController create(RunId runId, Iterable<LogHandler> logHandlers,

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
new file mode 100644
index 0000000..ee98352
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
@@ -0,0 +1,127 @@
+/*
+ * 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.twill.yarn;
+
+import org.apache.twill.api.AbstractTwillRunnable;
+import org.apache.twill.api.ResourceReport;
+import org.apache.twill.api.TwillApplication;
+import org.apache.twill.api.TwillController;
+import org.apache.twill.api.TwillRunResources;
+import org.apache.twill.api.TwillSpecification;
+import org.apache.twill.api.logging.PrinterLogHandler;
+import org.apache.twill.common.ServiceListenerAdapter;
+import org.apache.twill.common.Threads;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.PrintWriter;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests whether starting containers with debugging on works.
+ */
+public class DebugTestRun extends BaseYarnTest {
+
+  /**
+   * An application that contains two {@link DummyRunnable}.
+   */
+  public static final class DummyApplication implements TwillApplication {
+
+    @Override
+    public TwillSpecification configure() {
+      return TwillSpecification.Builder.with()
+                                       .setName("DummyApp")
+                                       .withRunnable()
+                                       .add("r1", new DummyRunnable()).noLocalFiles()
+                                       .add("r2", new DummyRunnable()).noLocalFiles()
+                                       .anyOrder()
+                                       .build();
+    }
+  }
+
+  /**
+   * A Runnable that will sleep in a loop until stopped.
+   */
+  public static final class DummyRunnable extends AbstractTwillRunnable {
+
+    private volatile Thread runThread;
+
+    @Override
+    public void run() {
+      this.runThread = Thread.currentThread();
+      while (!Thread.interrupted()) {
+        try {
+          TimeUnit.MILLISECONDS.sleep(100);
+        } catch (InterruptedException e) {
+          break;
+        }
+      }
+    }
+
+    @Override
+    public void stop() {
+      if (runThread != null) {
+        runThread.interrupt();
+      }
+    }
+  }
+
+  private boolean waitForDebugPort(TwillController controller, String runnable, int timeLimit)
+    throws InterruptedException {
+    long millis = 0;
+    while (millis < 1000 * timeLimit) {
+      ResourceReport report = controller.getResourceReport();
+      if (report == null || report.getRunnableResources(runnable) == null) {
+        continue;
+      }
+      for (TwillRunResources resources : report.getRunnableResources(runnable)) {
+        if (resources.getDebugPort() != null) {
+          return true;
+        }
+      }
+      TimeUnit.MILLISECONDS.sleep(100);
+      millis += 100;
+    }
+    return false;
+  }
+
+  @Test
+  public void testDebugPort() throws Exception {
+    YarnTwillRunnerService runner = (YarnTwillRunnerService) YarnTestUtils.getTwillRunner();
+    runner.enableDebugging("r1");
+    runner.startAndWait();
+
+    TwillController controller = runner.prepare(new DummyApplication())
+                                       .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out)))
+                                       .start();
+    final CountDownLatch running = new CountDownLatch(1);
+    controller.addListener(new ServiceListenerAdapter() {
+      @Override
+      public void running() {
+        running.countDown();
+      }
+    }, Threads.SAME_THREAD_EXECUTOR);
+
+    Assert.assertTrue(running.await(30, TimeUnit.SECONDS));
+    Assert.assertTrue(waitForDebugPort(controller, "r1", 30));
+    controller.stop().get(30, TimeUnit.SECONDS);
+    // Sleep a bit before exiting.
+    TimeUnit.SECONDS.sleep(2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-yarn/src/test/java/org/apache/twill/yarn/SessionExpireTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/SessionExpireTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/SessionExpireTestRun.java
index ec29061..f378606 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/SessionExpireTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/SessionExpireTestRun.java
@@ -49,7 +49,7 @@ import javax.management.QueryExp;
 import javax.management.StringValueExp;
 
 /**
- * Test for testing ZK session expire from AM container
+ * Test for testing ZK session expire from AM container.
  */
 public class SessionExpireTestRun extends BaseYarnTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fe5cd789/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
index 51b6abf..c981200 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
@@ -34,7 +34,8 @@ import org.junit.runners.Suite;
                       ProvisionTimeoutTestRun.class,
                       LogHandlerTestRun.class,
                       SessionExpireTestRun.class,
-                      ServiceDiscoveryTest.class
+                      ServiceDiscoveryTest.class,
+                      DebugTestRun.class
                     })
 public final class YarnTestSuite {
 


[42/50] [abbrv] git commit: (TWILL-71) move debug options from runner to preparer

Posted by ch...@apache.org.
(TWILL-71) move debug options from runner to preparer


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

Branch: refs/heads/site
Commit: 62e51ebb745220cb74c962927adca182ec24def3
Parents: 9da2d3f
Author: anew <an...@continuuity.com>
Authored: Fri Apr 11 23:41:42 2014 -0700
Committer: anew <an...@continuuity.com>
Committed: Fri Apr 11 23:41:42 2014 -0700

----------------------------------------------------------------------
 .../org/apache/twill/api/TwillPreparer.java     | 39 +++++++++++++++++
 .../apache/twill/yarn/YarnTwillPreparer.java    | 46 +++++++++++++++-----
 .../twill/yarn/YarnTwillRunnerService.java      | 38 +---------------
 .../org/apache/twill/yarn/DebugTestRun.java     |  2 +-
 .../org/apache/twill/yarn/LocalFileTestRun.java | 11 +----
 5 files changed, 77 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/62e51ebb/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java b/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
index b2a3ce2..6caad70 100644
--- a/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
+++ b/twill-api/src/main/java/org/apache/twill/api/TwillPreparer.java
@@ -45,6 +45,45 @@ public interface TwillPreparer {
   TwillPreparer setUser(String user);
 
   /**
+   * This methods sets the extra JVM options that will be passed to the java command line for every runnable
+   * of the application started through this {@link org.apache.twill.api.TwillPreparer} instance.
+   *
+   * This is intended for advance usage. All options will be passed unchanged to the java command line. Invalid
+   * options could cause application not able to start.
+   *
+   * @param options extra JVM options.
+   */
+  TwillPreparer setJVMOptions(String options);
+
+  /**
+   * This methods adds extra JVM options that will be passed to the java command line for every runnable
+   * of the application started through this {@link org.apache.twill.api.TwillPreparer} instance.
+   *
+   * This is intended for advance usage. All options will be passed unchanged to the java command line. Invalid
+   * options could cause application not able to start.
+   *
+   * @param options extra JVM options.
+   */
+  TwillPreparer addJVMOptions(String options);
+
+  /**
+   * Enable debugging for runnables, without suspending the virtual machine to wait for the debugger.
+   * This replaces any previous debug settings.
+   * @param runnables the names of runnables to enable for debugging. If empty, it means all runnables.
+   */
+  TwillPreparer enableDebugging(String ... runnables);
+
+  /**
+   * Enable debugging for runnables. This replaces any previous debug settings.
+   * @param doSuspend whether the virtual machines should be supended until the debugger connects. This
+   *                  option allows to debug a container from the very beginning. Note that in that case,
+   *                  the container cannot notify the controller of its debug port until the debugger is
+   *                  attached - you must figure out where it is running using the YARN console or APIs.
+   * @param runnables the names of runnables to enable for debugging. If empty, it means all runnables.
+   */
+  TwillPreparer enableDebugging(boolean doSuspend, String ... runnables);
+
+  /**
    * Sets the list of arguments that will be passed to the application. The arguments can be retrieved
    * from {@link TwillContext#getApplicationArguments()}.
    *

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/62e51ebb/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
index a02efea..2ed2de0 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillPreparer.java
@@ -21,7 +21,6 @@ import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicates;
-import com.google.common.base.Supplier;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.HashMultimap;
@@ -113,7 +112,6 @@ final class YarnTwillPreparer implements TwillPreparer {
   private final YarnAppClient yarnAppClient;
   private final ZKClient zkClient;
   private final LocationFactory locationFactory;
-  private final JvmOptions jvmOpts;
   private final YarnTwillControllerFactory controllerFactory;
   private final RunId runId;
 
@@ -126,22 +124,25 @@ final class YarnTwillPreparer implements TwillPreparer {
   private final Credentials credentials;
   private final int reservedMemory;
   private String user;
+  private String extraOptions;
+  private JvmOptions.DebugOptions debugOptions = JvmOptions.DebugOptions.NO_DEBUG;
 
-  YarnTwillPreparer(YarnConfiguration yarnConfig, TwillSpecification twillSpec, YarnAppClient yarnAppClient,
-                    ZKClient zkClient, LocationFactory locationFactory, Supplier<JvmOptions> jvmOpts,
+  YarnTwillPreparer(YarnConfiguration yarnConfig, TwillSpecification twillSpec,
+                    YarnAppClient yarnAppClient, ZKClient zkClient,
+                    LocationFactory locationFactory, String extraOptions,
                     YarnTwillControllerFactory controllerFactory) {
     this.yarnConfig = yarnConfig;
     this.twillSpec = twillSpec;
     this.yarnAppClient = yarnAppClient;
     this.zkClient = ZKClients.namespace(zkClient, "/" + twillSpec.getName());
     this.locationFactory = locationFactory;
-    this.jvmOpts = jvmOpts.get();
     this.controllerFactory = controllerFactory;
     this.runId = RunIds.generate();
     this.credentials = createCredentials();
     this.reservedMemory = yarnConfig.getInt(Configs.Keys.JAVA_RESERVED_MEMORY_MB,
                                             Configs.Defaults.JAVA_RESERVED_MEMORY_MB);
     this.user = System.getProperty("user.name");
+    this.extraOptions = extraOptions;
   }
 
   @Override
@@ -157,6 +158,29 @@ final class YarnTwillPreparer implements TwillPreparer {
   }
 
   @Override
+  public TwillPreparer setJVMOptions(String options) {
+    this.extraOptions = options;
+    return this;
+  }
+
+  @Override
+  public TwillPreparer addJVMOptions(String options) {
+    this.extraOptions = extraOptions == null ? options : extraOptions + " " + options;
+    return this;
+  }
+
+  @Override
+  public TwillPreparer enableDebugging(String... runnables) {
+    return enableDebugging(false, runnables);
+  }
+
+  @Override
+  public TwillPreparer enableDebugging(boolean doSuspend, String... runnables) {
+    this.debugOptions = new JvmOptions.DebugOptions(true, doSuspend, ImmutableSet.copyOf(runnables));
+    return this;
+  }
+
+  @Override
   public TwillPreparer withApplicationArguments(String... args) {
     return withApplicationArguments(ImmutableList.copyOf(args));
   }
@@ -242,7 +266,7 @@ final class YarnTwillPreparer implements TwillPreparer {
           saveSpecification(twillSpec, runnableLocalFiles, localFiles);
           saveLogback(localFiles);
           saveLauncher(localFiles);
-          saveJvmOptions(jvmOpts, localFiles);
+          saveJvmOptions(localFiles);
           saveArguments(new Arguments(arguments, runnableArgs), localFiles);
           saveLocalFiles(localFiles, ImmutableSet.of(Constants.Files.TWILL_SPEC,
                                                      Constants.Files.LOGBACK_TEMPLATE,
@@ -275,7 +299,7 @@ final class YarnTwillPreparer implements TwillPreparer {
               "-Dtwill.app=$" + EnvKeys.TWILL_APP_NAME,
               "-cp", Constants.Files.LAUNCHER_JAR + ":$HADOOP_CONF_DIR",
               "-Xmx" + (Constants.APP_MASTER_MEMORY_MB - Constants.APP_MASTER_RESERVED_MEMORY_MB) + "m",
-              jvmOpts.getExtraOptions() == null ? "" : jvmOpts.getExtraOptions(),
+              extraOptions == null ? "" : extraOptions,
               TwillLauncher.class.getName(),
               Constants.Files.APP_MASTER_JAR,
               ApplicationMasterMain.class.getName(),
@@ -499,15 +523,15 @@ final class YarnTwillPreparer implements TwillPreparer {
     localFiles.put(Constants.Files.LAUNCHER_JAR, createLocalFile(Constants.Files.LAUNCHER_JAR, location));
   }
 
-  private void saveJvmOptions(JvmOptions opts, Map<String, LocalFile> localFiles) throws IOException {
-    if ((opts.getExtraOptions() == null || opts.getExtraOptions().isEmpty()) &&
-      JvmOptions.DebugOptions.NO_DEBUG.equals(opts.getDebugOptions())) {
+  private void saveJvmOptions(Map<String, LocalFile> localFiles) throws IOException {
+    if ((extraOptions == null || extraOptions.isEmpty()) &&
+      JvmOptions.DebugOptions.NO_DEBUG.equals(debugOptions)) {
       // If no vm options, no need to localize the file.
       return;
     }
     LOG.debug("Create and copy {}", Constants.Files.JVM_OPTIONS);
     final Location location = createTempLocation(Constants.Files.JVM_OPTIONS);
-    JvmOptionsCodec.encode(opts, new OutputSupplier<Writer>() {
+    JvmOptionsCodec.encode(new JvmOptions(extraOptions, debugOptions), new OutputSupplier<Writer>() {
       @Override
       public Writer getOutput() throws IOException {
         return new OutputStreamWriter(location.getOutputStream(), Charsets.UTF_8);

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/62e51ebb/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
index 4ecfb0e..dadedce 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
@@ -21,7 +21,6 @@ import com.google.common.base.Charsets;
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
-import com.google.common.base.Suppliers;
 import com.google.common.base.Throwables;
 import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.HashMultimap;
@@ -65,7 +64,6 @@ import org.apache.twill.filesystem.HDFSLocationFactory;
 import org.apache.twill.filesystem.Location;
 import org.apache.twill.filesystem.LocationFactory;
 import org.apache.twill.internal.Constants;
-import org.apache.twill.internal.JvmOptions;
 import org.apache.twill.internal.ProcessController;
 import org.apache.twill.internal.RunIds;
 import org.apache.twill.internal.SingleRunnableApplication;
@@ -135,7 +133,6 @@ public final class YarnTwillRunnerService extends AbstractIdleService implements
   private Cancellable watchCancellable;
 
   private volatile String jvmOptions = null;
-  private volatile JvmOptions.DebugOptions debugOptions = JvmOptions.DebugOptions.NO_DEBUG;
 
   public YarnTwillRunnerService(YarnConfiguration config, String zkConnect) {
     this(config, zkConnect, new HDFSLocationFactory(getFileSystem(config), "/twill"));
@@ -172,38 +169,6 @@ public final class YarnTwillRunnerService extends AbstractIdleService implements
     return jvmOptions;
   }
 
-  /**
-   * Disable debugging for runnables. It only affects applications that are started after this
-   * method is called. Note that debugging is disabled by default, and this method is only needed
-   * to revert a previous call of @link #enableDebugging.
-   */
-  public void disableDebugging() {
-    this.debugOptions = JvmOptions.DebugOptions.NO_DEBUG;
-  }
-
-  /**
-   * Enable debugging for runnables, without suspending the virtual machine to wait for the debugger.
-   * This replaces any previous debug settings and only affects applications that are started after
-   * this method is called.
-   * @param runnables the names of runnables to enable for debugging.
-   */
-  public void enableDebugging(String ... runnables) {
-    this.debugOptions = new JvmOptions.DebugOptions(true, false, ImmutableSet.copyOf(runnables));
-  }
-
-  /**
-   * Enable debugging for runnables. This replaces any previous debug settings and only affects
-   * applications that are started after this method is called.
-   * @param doSuspend whether the virtual machines should be supended until the debugger connects. This
-   *                  option allows to debug a container from the very beginning. Note that in that case,
-   *                  the container cannot notify the controller of its debug port until the debugger is
-   *                  attached - you must figure out where it is running using the YARN console or APIs.
-   * @param runnables the names of runnables to enable for debugging.
-   */
-  public void enableDebugging(boolean doSuspend, String ... runnables) {
-    this.debugOptions = new JvmOptions.DebugOptions(true, false, ImmutableSet.copyOf(runnables));
-  }
-
   @Override
   public Cancellable scheduleSecureStoreUpdate(final SecureStoreUpdater updater,
                                                long initialDelay, long delay, TimeUnit unit) {
@@ -273,8 +238,7 @@ public final class YarnTwillRunnerService extends AbstractIdleService implements
     final TwillSpecification twillSpec = application.configure();
     final String appName = twillSpec.getName();
 
-    return new YarnTwillPreparer(yarnConfig, twillSpec, yarnAppClient, zkClientService, locationFactory,
-                                 Suppliers.ofInstance(new JvmOptions(jvmOptions, debugOptions)),
+    return new YarnTwillPreparer(yarnConfig, twillSpec, yarnAppClient, zkClientService, locationFactory, jvmOptions,
                                  new YarnTwillControllerFactory() {
       @Override
       public YarnTwillController create(RunId runId, Iterable<LogHandler> logHandlers,

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/62e51ebb/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
index ee98352..daf0bc1 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
@@ -104,10 +104,10 @@ public class DebugTestRun extends BaseYarnTest {
   @Test
   public void testDebugPort() throws Exception {
     YarnTwillRunnerService runner = (YarnTwillRunnerService) YarnTestUtils.getTwillRunner();
-    runner.enableDebugging("r1");
     runner.startAndWait();
 
     TwillController controller = runner.prepare(new DummyApplication())
+                                       .enableDebugging("r1")
                                        .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out)))
                                        .start();
     final CountDownLatch running = new CountDownLatch(1);

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/62e51ebb/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
index 78f61a9..2bf0b5a 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/LocalFileTestRun.java
@@ -57,23 +57,14 @@ public final class LocalFileTestRun extends BaseYarnTest {
                                         Charsets.UTF_8);
 
     TwillRunner runner = YarnTestUtils.getTwillRunner();
-    String prevJVMOptions = "";
-    if (runner instanceof YarnTwillRunnerService) {
-      YarnTwillRunnerService yarnRunner = (YarnTwillRunnerService) runner;
-      prevJVMOptions = yarnRunner.getJVMOptions() != null ? yarnRunner.getJVMOptions() : "";
-      yarnRunner.setJVMOptions(prevJVMOptions + " -verbose:gc -Xloggc:gc.log -XX:+PrintGCDetails");
-    }
 
     TwillController controller = runner.prepare(new LocalFileApplication())
+      .addJVMOptions(" -verbose:gc -Xloggc:gc.log -XX:+PrintGCDetails")
       .withApplicationArguments("local")
       .withArguments("LocalFileSocketServer", "local2")
       .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
       .start();
 
-    if (runner instanceof YarnTwillRunnerService) {
-      ((YarnTwillRunnerService) runner).setJVMOptions(prevJVMOptions);
-    }
-
     Iterable<Discoverable> discoverables = controller.discoverService("local");
     Assert.assertTrue(YarnTestUtils.waitForSize(discoverables, 1, 60));
 


[44/50] [abbrv] git commit: (TWILL-72) fix enableDebugging() with empty list of runnables

Posted by ch...@apache.org.
(TWILL-72) fix enableDebugging() with empty list of runnables


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

Branch: refs/heads/site
Commit: 1187ebf790bc836e218b19b00854102e15047f73
Parents: 62e51eb
Author: anew <an...@continuuity.com>
Authored: Mon Apr 14 12:12:41 2014 -0700
Committer: anew <an...@continuuity.com>
Committed: Mon Apr 14 12:12:41 2014 -0700

----------------------------------------------------------------------
 pom.xml                                         |  2 +-
 twill-api/pom.xml                               |  2 +-
 twill-common/pom.xml                            |  2 +-
 twill-core/pom.xml                              |  2 +-
 .../org/apache/twill/internal/JvmOptions.java   |  4 ++-
 .../twill/internal/json/JvmOptionsCodec.java    |  9 ++++---
 .../internal/json/JvmOptionsCodecTest.java      | 25 ++++++++++++++++++
 twill-discovery-api/pom.xml                     |  2 +-
 twill-discovery-core/pom.xml                    |  2 +-
 twill-examples/pom.xml                          |  2 +-
 twill-ext/pom.xml                               |  2 +-
 twill-yarn/pom.xml                              |  2 +-
 .../org/apache/twill/yarn/DebugTestRun.java     | 27 +++++++++++++++++++-
 twill-zookeeper/pom.xml                         |  2 +-
 14 files changed, 70 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b97bac1..1481d54 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
 
     <groupId>org.apache.twill</groupId>
     <artifactId>twill-parent</artifactId>
-    <version>0.2.0-incubating-SNAPSHOT</version>
+    <version>0.3.0-incubating-SNAPSHOT</version>
     <packaging>pom</packaging>
     <name>Apache Twill</name>
     <url>http://twill.incubator.apache.org</url>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-api/pom.xml b/twill-api/pom.xml
index 338a716..250289c 100644
--- a/twill-api/pom.xml
+++ b/twill-api/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.twill</groupId>
         <artifactId>twill-parent</artifactId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
 
     <artifactId>twill-api</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-common/pom.xml
----------------------------------------------------------------------
diff --git a/twill-common/pom.xml b/twill-common/pom.xml
index 7506d84..e9e641f 100644
--- a/twill-common/pom.xml
+++ b/twill-common/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-core/pom.xml b/twill-core/pom.xml
index 124547c..7b7a085 100644
--- a/twill-core/pom.xml
+++ b/twill-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-core/src/main/java/org/apache/twill/internal/JvmOptions.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/JvmOptions.java b/twill-core/src/main/java/org/apache/twill/internal/JvmOptions.java
index a260f5b..f827433 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/JvmOptions.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/JvmOptions.java
@@ -18,6 +18,7 @@
 package org.apache.twill.internal;
 
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
 
 import java.util.Set;
 
@@ -60,7 +61,8 @@ public final class JvmOptions {
     public DebugOptions(boolean doDebug, boolean doSuspend, Iterable<String> runnables) {
       this.doDebug = doDebug;
       this.doSuspend = doDebug && doSuspend;
-      this.runnables = doDebug && runnables != null ? ImmutableSet.copyOf(runnables) : null;
+      this.runnables =
+        doDebug && runnables != null && Iterables.size(runnables) > 0 ? ImmutableSet.copyOf(runnables) : null;
     }
 
     public boolean doDebug() {

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-core/src/main/java/org/apache/twill/internal/json/JvmOptionsCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/JvmOptionsCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/JvmOptionsCodec.java
index a1b7eb9..09fc91b 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/JvmOptionsCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/JvmOptionsCodec.java
@@ -96,8 +96,9 @@ public class JvmOptionsCodec implements JsonSerializer<JvmOptions>, JsonDeserial
         return JvmOptions.DebugOptions.NO_DEBUG;
       }
       Boolean doSuspend = context.deserialize(jsonObj.get("doSuspend"), Boolean.class);
-      Set<String> runnables = context.deserialize(jsonObj.get("runnables"), new TypeToken<Set<String>>() { }.getType());
-      return new JvmOptions.DebugOptions(true, doSuspend, ImmutableSet.copyOf(runnables));
+      Set<String> runnables = context.deserialize(jsonObj.get("runnables"),
+                                                  new TypeToken<Set<String>>() { }.getType());
+      return new JvmOptions.DebugOptions(true, doSuspend, runnables == null ? null : ImmutableSet.copyOf(runnables));
     }
 
     @Override
@@ -105,7 +106,9 @@ public class JvmOptionsCodec implements JsonSerializer<JvmOptions>, JsonDeserial
       JsonObject json = new JsonObject();
       json.add("doDebug", context.serialize(src.doDebug()));
       json.add("doSuspend", context.serialize(src.doSuspend()));
-      json.add("runnables", context.serialize(src.getRunnables()));
+      if (src.getRunnables() != null) {
+        json.add("runnables", context.serialize(src.getRunnables()));
+      }
       return json;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-core/src/test/java/org/apache/twill/internal/json/JvmOptionsCodecTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/json/JvmOptionsCodecTest.java b/twill-core/src/test/java/org/apache/twill/internal/json/JvmOptionsCodecTest.java
index 66187f2..2791e72 100644
--- a/twill-core/src/test/java/org/apache/twill/internal/json/JvmOptionsCodecTest.java
+++ b/twill-core/src/test/java/org/apache/twill/internal/json/JvmOptionsCodecTest.java
@@ -29,6 +29,8 @@ import java.io.Reader;
 import java.io.StringReader;
 import java.io.StringWriter;
 import java.io.Writer;
+import java.util.Collections;
+import java.util.List;
 
 /**
  * Tests the JvmOptions Codec.
@@ -79,4 +81,27 @@ public class JvmOptionsCodecTest {
     Assert.assertEquals(options.getDebugOptions().doSuspend(), options1.getDebugOptions().doSuspend());
     Assert.assertEquals(options.getDebugOptions().getRunnables(), options1.getDebugOptions().getRunnables());
   }
+
+  @Test
+  public void testNoRunnables() throws Exception {
+    List<String> noRunnables = Collections.emptyList();
+    JvmOptions options = new JvmOptions(null, new JvmOptions.DebugOptions(true, false, noRunnables));
+    final StringWriter writer = new StringWriter();
+    JvmOptionsCodec.encode(options, new OutputSupplier<Writer>() {
+      @Override
+      public Writer getOutput() throws IOException {
+        return writer;
+      }
+    });
+    JvmOptions options1 = JvmOptionsCodec.decode(new InputSupplier<Reader>() {
+      @Override
+      public Reader getInput() throws IOException {
+        return new StringReader(writer.toString());
+      }
+    });
+    Assert.assertEquals(options.getExtraOptions(), options1.getExtraOptions());
+    Assert.assertEquals(options.getDebugOptions().doDebug(), options1.getDebugOptions().doDebug());
+    Assert.assertEquals(options.getDebugOptions().doSuspend(), options1.getDebugOptions().doSuspend());
+    Assert.assertEquals(options.getDebugOptions().getRunnables(), options1.getDebugOptions().getRunnables());
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-discovery-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-api/pom.xml b/twill-discovery-api/pom.xml
index 2a12f39..0ce6c38 100644
--- a/twill-discovery-api/pom.xml
+++ b/twill-discovery-api/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-discovery-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-core/pom.xml b/twill-discovery-core/pom.xml
index 5570986..2bdad6b 100644
--- a/twill-discovery-core/pom.xml
+++ b/twill-discovery-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-examples/pom.xml
----------------------------------------------------------------------
diff --git a/twill-examples/pom.xml b/twill-examples/pom.xml
index 60f48fc..79a9581 100644
--- a/twill-examples/pom.xml
+++ b/twill-examples/pom.xml
@@ -23,7 +23,7 @@ limitations under the License.
     <modelVersion>4.0.0</modelVersion>
     <groupId>org.apache.twill.example</groupId>
     <artifactId>twill-examples</artifactId>
-    <version>0.2.0-incubating-SNAPSHOT</version>
+    <version>0.3.0-incubating-SNAPSHOT</version>
     <name>Twill examples</name>
     <packaging>pom</packaging>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-ext/pom.xml
----------------------------------------------------------------------
diff --git a/twill-ext/pom.xml b/twill-ext/pom.xml
index d6245bf..3d835bf 100644
--- a/twill-ext/pom.xml
+++ b/twill-ext/pom.xml
@@ -22,7 +22,7 @@ limitations under the License.
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
 
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-yarn/pom.xml b/twill-yarn/pom.xml
index de45335..014489c 100644
--- a/twill-yarn/pom.xml
+++ b/twill-yarn/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
index daf0bc1..0cb6fc8 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/DebugTestRun.java
@@ -102,7 +102,7 @@ public class DebugTestRun extends BaseYarnTest {
   }
 
   @Test
-  public void testDebugPort() throws Exception {
+  public void testDebugPortOneRunnable() throws Exception {
     YarnTwillRunnerService runner = (YarnTwillRunnerService) YarnTestUtils.getTwillRunner();
     runner.startAndWait();
 
@@ -124,4 +124,29 @@ public class DebugTestRun extends BaseYarnTest {
     // Sleep a bit before exiting.
     TimeUnit.SECONDS.sleep(2);
   }
+
+  @Test
+  public void testDebugPortAllRunnables() throws Exception {
+    YarnTwillRunnerService runner = (YarnTwillRunnerService) YarnTestUtils.getTwillRunner();
+    runner.startAndWait();
+
+    TwillController controller = runner.prepare(new DummyApplication())
+                                       .enableDebugging()
+                                       .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out)))
+                                       .start();
+    final CountDownLatch running = new CountDownLatch(1);
+    controller.addListener(new ServiceListenerAdapter() {
+      @Override
+      public void running() {
+        running.countDown();
+      }
+    }, Threads.SAME_THREAD_EXECUTOR);
+
+    Assert.assertTrue(running.await(30, TimeUnit.SECONDS));
+    Assert.assertTrue(waitForDebugPort(controller, "r1", 30));
+    Assert.assertTrue(waitForDebugPort(controller, "r2", 30));
+    controller.stop().get(30, TimeUnit.SECONDS);
+    // Sleep a bit before exiting.
+    TimeUnit.SECONDS.sleep(2);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/1187ebf7/twill-zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/twill-zookeeper/pom.xml b/twill-zookeeper/pom.xml
index 70de6bf..8b1f4fe 100644
--- a/twill-zookeeper/pom.xml
+++ b/twill-zookeeper/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.2.0-incubating-SNAPSHOT</version>
+        <version>0.3.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 


[14/50] [abbrv] git commit: Move KEYS file to dist.apache.org

Posted by ch...@apache.org.
Move KEYS file to dist.apache.org

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

Branch: refs/heads/site
Commit: bbdb21215ca6c30659d6e51a9d79bb28cbb67e23
Parents: 95b6c64
Author: Terence Yim <ch...@apache.org>
Authored: Mon Jan 27 23:33:35 2014 -0800
Committer: Terence Yim <ch...@apache.org>
Committed: Mon Jan 27 23:33:35 2014 -0800

----------------------------------------------------------------------
 KEYS | 61 -------------------------------------------------------------
 1 file changed, 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/bbdb2121/KEYS
----------------------------------------------------------------------
diff --git a/KEYS b/KEYS
deleted file mode 100644
index 880bd27..0000000
--- a/KEYS
+++ /dev/null
@@ -1,61 +0,0 @@
-This file contains the PGP keys of various developers.
-
-Users: pgp < KEYS
-       gpg --import KEYS
-Developers: 
-        pgp -kxa <your name> and append it to this file.
-        (pgpk -ll <your name> && pgpk -xa <your name>) >> this file.
-        (gpg --list-sigs <your name>
-             && gpg --armor --export <your name>) >> this file.
-
-Type bits/keyID     Date       User ID
-pub  4096R/2167BDB4 2014-01-27 Terence Yim (CODE SIGNING KEY) <ch...@apache.org>
-
------BEGIN PGP PUBLIC KEY BLOCK-----
-Version: SKS 1.1.4
-Comment: Hostname: pgp.mit.edu
-
-mQINBFLm1Z8BEADthx0oPr1A25xZ0lvdBOcqg6cPJG1UIyIBWojOrles03EBohq9p/epoiwi
-RGI2cbChLHUNEOl4fyDbBQXQsVDzAfICSjBgiqTwf7WxNH6kO1HFfV32tA6aszqBOv4iF8L6
-/GYotrf/crQgu2nc46oYuEfkKSlsroRzJxcT73vX7E1mZ+pcNLm+ZpKccT2pMZ+P3CN9m7dt
-fqYOATYp5eM9c0/RsRjg02BPOl2LAc5SRrf7jTwHJRTpqRueenxujHqT+oSIiSMOYfjGc56Q
-y9BKsBsMraM9DdccNpjvF1vlefONPXO8p3lvfxdC5pjvBuuROo64HmaR3RpGJ/4PS9RYgYkW
-TS3G9dpQM2PyE5kJ8cvESVn6YRAuR2mSQDUQcEYYG6qU2czFkzXoUBzu/Uj2iIVqZFhaS496
-ECz+3vokaGhXgESmXNaCzBpQs4SJtgANfnlF4TlnKifbbPtTyJdCnB8R/7ymd875CwFAoFgk
-Q7Ksxnb18NQJ00p0DgSV4pGtVHmz3S1dMmRqN/9w1lRccwwI7GDkSNqWciWw7DeJHJKEDUCw
-zVRpC3s38LZNscj1W6rKhCkx43EZYORAEGqVKtql6q/8SBYKmtoUAeeMPLm6akzRZBLqApyo
-Wai6zSVxbGX8e58M419/X5O/maUpohebr+apBug5TOcq5U24TwARAQABtDJUZXJlbmNlIFlp
-bSAoQ09ERSBTSUdOSU5HIEtFWSkgPGNodHlpbUBhcGFjaGUub3JnPokCNwQTAQIAIQIbAwIe
-AQIXgAUCUubWOQULCQgHAwUVCgkICwUWAgMBAAAKCRDKlsLqIWe9tHy6D/9GLQjklfAcThae
-mbe+P1GMXBSiWAaxPvRj+lfYs1NMamqgweeiGLfLWCpi5l90J1u9wya0RxCBxPYz1Ad5zb8u
-JapQzh+CcVy2ggm4CjZmlnOzLKtnZhRVIHK2O1O44HtJNdUcxJQhPvNB9qWjGKvX8e0WISml
-LLYyWvqp7jVwyaTivP87ye7n07M6A/sKFWob/mUOx/4GqEWLPTZ8Qi3v/UsAqElfk4NfaZM+
-Qc+xH17RlBo5CD9+UwD27loPV25N9PdEWOahFTR0a1VazojXiGzD/RS2srnytddK5dDe/hCy
-JNXZ4N8zoiCYxT9oSxK/UYeB9xe92mLf2YwcxYjV6sztOf1xqhEx/xFh7Yq5B3URKL0j2FA8
-Pw2H2mX+wcwZXoyfgr6QPtzAHm+ckMHVf6DNVKIWoPVwI6FKGlbK6gMxp40OLT+eYXt8/AP7
-uH6e8WJf5es4Nz4WFZ5ssIFYK/CsHOe3/7NR3s90Pszdk2ODwbEsJdHlhX1h8Uun4tH2Sj29
-gWhUGyse2bdxFfuyWzZedrUyC/4TSTgYycwmHKuviMBgmr6BQjV3IstV9MzorhNfUEx3KSJd
-HK3C/uggNvkSpUOFdyeJvt3s0Ntv7ACZvboFgb0SOXYwH1csdSObNIDamsjM3uyjiQnMc6O4
-wrEG11LCVgYxOjAc7BxqPbkCDQRS5tWfARAAuyY6r/fc8Ff7qaV6yrDoTfGc+ZRtgfgIvhIr
-qadyQFo+4s76YmV0n+8kmvzxn3ar/w+InA0rA/z9x2Cn40qgtm/CGPPJN+RL79Fm+9Ot2ZRJ
-xjBFnE0upZbvNFttgkjiBdAyLjsrxl+pcIerFhwP/l0jF4SAWB1FwYi3aZhIwZG9i27nkVsa
-neQivLFT3aZkNALHXiapHGOKx7iTf9St0UA7YVyS5ucVS0U50wqmxzn5NaWHZNF2qcHNgrlk
-N+cZIUQVccvdY6Ho/DcZ28+9msieL5GDVOnm0dl0OyFa0/i3oe12GcH1I7UDQzAm5uSUy2Jf
-pDjfXap+462n3KIo/ZktDKa+RxMFUjDZXdEAzlFQ4BeTZpuhwsAsghJ0BZxGlNCZ0n/gGzfn
-wah1ihio4mIhT8MpPnntCz9rfYNbxa8LNuExKQ/GA+h3jSH1E1w/y0zhewz5OlGNAAC4UrwV
-8y3oN4jxhqzNl4eRA3dgSlFq+tF/N2gpVDwjvmldgpcSiaVIvSFplCUbUs7vwh73qta+cJSr
-kHiNkbyLuMc3tzssvMLs+EOaJzCZlzNi0CZbiXFSp9zUctOXfqkGTcgdlK3mXspITEaJgmmU
-GL2R/xWwTiN1YHrXk3D4Etb5eZcuVm7N0a6fPHnlNuJ8JeyPei9BDXyHhtJ91wgQg/uMCmUA
-EQEAAYkCHwQYAQIACQUCUubVnwIbDAAKCRDKlsLqIWe9tDMkEADJs2/c1WxMsD/hynlSssp1
-OMidvB2aWuNZBYuBjP3ZllJYGSEZbXjwPUEb3vMUNk7xEzV/3niHEWTpEb3SpC47pSiW5aw3
-VhJIZXAT9crOILN3Bk+oJUYOPwFNxogYBnN5Rs5DoPhGtzzdb1wPwB+FXFum/+HVvXR6XotR
-AQr44PrvmC7Y5RTqJ3N8r3H+2vqcJ89FPuolxdAJ3OZ+y3L8d5ejK60+sjlPkpMYq5plbWfY
-qiyV3xzahFOH7rWVpANyvJnApp99vFSSVcev6G40U6LAcuJ1fdqlWSF0TU+iEwvdY7S5fCMO
-+6s4y4+CNZJw045KP5ZniHJQmOLVGyxU9UKrjHGdmR2v/qodUCKYwONrO3vGGtyblWX70Htm
-JCzpUXzEIKUFESiyLEBzYRmuI1GV/9p/ZNX9nqWSZIrsWmVIe+Y8IAJIyP36SR5iSku0T0OB
-eyzhhD7KvN0dTl0A1XpJTr5xZXWUeig7v2WBS30D6npE/Q9axC4Y1THa2HpZWcsyIVXf3X+v
-rdKlrKaSSQzaAcggEMhsAxjGoIme1K32pNl3mhMvLdNGFTRTonw0o62HbJFEaX+E+HzbV0Ap
-o+BXfVQlGDwy1o+ZbDXQ/BcdXyj5gcjKhmqdXv3u3hrr3JnxBnbORVJ9KEq0Raxnl8NUz7GJ
-HRgYfJQv18t0mA==
-=lr4q
------END PGP PUBLIC KEY BLOCK-----


[41/50] [abbrv] git commit: (TWILL-68) Implements the stop() method in the AbstractTwillRunnable to simplify children class implementation.

Posted by ch...@apache.org.
(TWILL-68) Implements the stop() method in the AbstractTwillRunnable to simplify children class implementation.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 9da2d3fec9767206912b3e894ed4c2ed5e1383f6
Parents: fe5cd78
Author: Terence Yim <te...@continuuity.com>
Authored: Wed Apr 9 16:38:45 2014 -0700
Committer: Terence Yim <te...@continuuity.com>
Committed: Fri Apr 11 12:13:29 2014 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/twill/api/AbstractTwillRunnable.java   | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9da2d3fe/twill-api/src/main/java/org/apache/twill/api/AbstractTwillRunnable.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/AbstractTwillRunnable.java b/twill-api/src/main/java/org/apache/twill/api/AbstractTwillRunnable.java
index 67cec0a..614099d 100644
--- a/twill-api/src/main/java/org/apache/twill/api/AbstractTwillRunnable.java
+++ b/twill-api/src/main/java/org/apache/twill/api/AbstractTwillRunnable.java
@@ -57,6 +57,11 @@ public abstract class AbstractTwillRunnable implements TwillRunnable {
   }
 
   @Override
+  public void stop() {
+    // No-op by default. Left for children class to override.
+  }
+
+  @Override
   public void destroy() {
     // No-op by default. Left for children class to override.
   }


[26/50] [abbrv] git commit: [TWILL-43] Fix Kafka log appender to retry on publish failure.

Posted by ch...@apache.org.
[TWILL-43] Fix Kafka log appender to retry on publish failure.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 146740b1289b7610c3eaa3c07cdaf322cea8e97a
Parents: 9f66b86
Author: Terence Yim <te...@continuuity.com>
Authored: Tue Feb 11 14:07:19 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Wed Feb 12 14:21:52 2014 -0800

----------------------------------------------------------------------
 .../apache/twill/internal/json/JsonUtils.java   |   2 +-
 .../kafka/client/SimpleKafkaPublisher.java      |  14 ++-
 .../twill/internal/logging/KafkaAppender.java   | 114 ++++++++++++++-----
 .../apache/twill/yarn/LogHandlerTestRun.java    |  82 +++++++++++++
 .../org/apache/twill/yarn/YarnTestSuite.java    |   3 +-
 5 files changed, 177 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/146740b1/twill-core/src/main/java/org/apache/twill/internal/json/JsonUtils.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/JsonUtils.java b/twill-core/src/main/java/org/apache/twill/internal/json/JsonUtils.java
index 9556ad8..7e6a11b 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/JsonUtils.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/JsonUtils.java
@@ -33,7 +33,7 @@ public final class JsonUtils {
    */
   public static String getAsString(JsonObject json, String property) {
     JsonElement jsonElement = json.get(property);
-    if (jsonElement.isJsonNull()) {
+    if (jsonElement == null || jsonElement.isJsonNull()) {
       return null;
     }
     if (jsonElement.isJsonPrimitive()) {

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/146740b1/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaPublisher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaPublisher.java b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaPublisher.java
index 97b14c6..302a6a4 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaPublisher.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/kafka/client/SimpleKafkaPublisher.java
@@ -90,11 +90,15 @@ final class SimpleKafkaPublisher implements KafkaPublisher {
 
     @Override
     public ListenableFuture<Integer> send() {
-      int size = messages.size();
-      producer.send(messages);
-
-      messages.clear();
-      return Futures.immediateFuture(size);
+      try {
+        int size = messages.size();
+        producer.send(messages);
+        return Futures.immediateFuture(size);
+      } catch (Exception e) {
+        return Futures.immediateFailedFuture(e);
+      } finally {
+        messages.clear();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/146740b1/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java b/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
index 9a22bea..8345865 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/logging/KafkaAppender.java
@@ -27,11 +27,14 @@ import ch.qos.logback.classic.spi.StackTraceElementProxy;
 import ch.qos.logback.core.AppenderBase;
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
 import com.google.common.base.Throwables;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.Service;
 import com.google.gson.stream.JsonWriter;
 import org.apache.twill.common.Services;
 import org.apache.twill.common.Threads;
@@ -48,11 +51,16 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.io.StringWriter;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -141,22 +149,24 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
                                  RetryStrategies.fixDelay(1, TimeUnit.SECONDS))));
 
     kafkaClient = new ZKKafkaClientService(zkClientService);
-    Futures.addCallback(Services.chainStart(zkClientService, kafkaClient), new FutureCallback<Object>() {
+    Futures.addCallback(Services.chainStart(zkClientService, kafkaClient),
+                        new FutureCallback<List<ListenableFuture<Service.State>>>() {
       @Override
-      public void onSuccess(Object result) {
+      public void onSuccess(List<ListenableFuture<Service.State>> result) {
+        for (ListenableFuture<Service.State> future : result) {
+          Preconditions.checkState(Futures.getUnchecked(future) == Service.State.RUNNING,
+                                   "Service is not running.");
+        }
         LOG.info("Kafka client started: " + zkConnectStr);
-        KafkaPublisher.Preparer preparer = kafkaClient.getPublisher(KafkaPublisher.Ack.LEADER_RECEIVED,
-                                                                    Compression.SNAPPY).prepare(topic);
-        publisher.set(preparer);
         scheduler.scheduleWithFixedDelay(flushTask, 0, flushPeriod, TimeUnit.MILLISECONDS);
       }
 
       @Override
       public void onFailure(Throwable t) {
         // Fail to talk to kafka. Other than logging, what can be done?
-        LOG.error("Failed to start kafka client.", t);
+        LOG.error("Failed to start kafka appender.", t);
       }
-    });
+    }, Threads.SAME_THREAD_EXECUTOR);
 
     super.start();
   }
@@ -170,7 +180,7 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
 
   public void forceFlush() {
     try {
-      publishLogs().get(2, TimeUnit.SECONDS);
+      publishLogs(2, TimeUnit.SECONDS);
     } catch (Exception e) {
       LOG.error("Failed to publish last batch of log.", e);
     }
@@ -185,24 +195,71 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
     }
   }
 
-  private ListenableFuture<Integer> publishLogs() {
-    // If the publisher is not available, simply returns a completed future.
-    KafkaPublisher.Preparer publisher = KafkaAppender.this.publisher.get();
-    if (publisher == null) {
-      return Futures.immediateFuture(0);
-    }
+  /**
+   * Publishes buffered logs to Kafka, within the given timeout.
+   *
+   * @return Number of logs published.
+   * @throws TimeoutException If timeout reached before publish completed.
+   */
+  private int publishLogs(long timeout, TimeUnit timeoutUnit) throws TimeoutException {
+    List<ByteBuffer> logs = Lists.newArrayListWithExpectedSize(bufferedSize.get());
 
-    int count = 0;
     for (String json : Iterables.consumingIterable(buffer)) {
-      publisher.add(Charsets.UTF_8.encode(json), 0);
-      count++;
+      logs.add(Charsets.UTF_8.encode(json));
+    }
+
+    long backOffTime = timeoutUnit.toNanos(timeout) / 10;
+    if (backOffTime <= 0) {
+      backOffTime = 1;
+    }
+
+    try {
+      Stopwatch stopwatch = new Stopwatch();
+      stopwatch.start();
+      long publishTimeout = timeout;
+
+      do {
+        try {
+          int published = doPublishLogs(logs).get(publishTimeout, timeoutUnit);
+          bufferedSize.addAndGet(-published);
+          return published;
+        } catch (ExecutionException e) {
+          LOG.error("Failed to publish logs to Kafka.", e);
+          TimeUnit.NANOSECONDS.sleep(backOffTime);
+          publishTimeout -= stopwatch.elapsedTime(timeoutUnit);
+          stopwatch.reset();
+          stopwatch.start();
+        }
+      } while (publishTimeout > 0);
+    } catch (InterruptedException e) {
+      LOG.warn("Logs publish to Kafka interrupted.", e);
     }
+    return 0;
+  }
+
+  private ListenableFuture<Integer> doPublishLogs(Collection <ByteBuffer> logs) {
     // Nothing to publish, simply returns a completed future.
-    if (count == 0) {
+    if (logs.isEmpty()) {
       return Futures.immediateFuture(0);
     }
 
-    bufferedSize.set(0);
+    // If the publisher is not available, tries to create one.
+    KafkaPublisher.Preparer publisher = KafkaAppender.this.publisher.get();
+    if (publisher == null) {
+      try {
+        KafkaPublisher.Preparer preparer = kafkaClient.getPublisher(KafkaPublisher.Ack.LEADER_RECEIVED,
+                                                                    Compression.SNAPPY).prepare(topic);
+        KafkaAppender.this.publisher.compareAndSet(null, preparer);
+        publisher = KafkaAppender.this.publisher.get();
+      } catch (Exception e) {
+        return Futures.immediateFailedFuture(e);
+      }
+    }
+
+    for (ByteBuffer buffer : logs) {
+      publisher.add(buffer, 0);
+    }
+
     return publisher.send();
   }
 
@@ -214,19 +271,14 @@ public final class KafkaAppender extends AppenderBase<ILoggingEvent> {
     return new Runnable() {
       @Override
       public void run() {
-        Futures.addCallback(publishLogs(), new FutureCallback<Integer>() {
-          @Override
-          public void onSuccess(Integer result) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Log entries published, size=" + result);
-            }
-          }
-
-          @Override
-          public void onFailure(Throwable t) {
-            LOG.error("Failed to push logs to kafka. Log entries dropped.", t);
+        try {
+          int published = publishLogs(2L, TimeUnit.SECONDS);
+          if (LOG.isDebugEnabled()) {
+            LOG.info("Published {} log messages to Kafka.", published);
           }
-        });
+        } catch (Exception e) {
+          LOG.error("Failed to push logs to Kafka. Log entries dropped.", e);
+        }
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/146740b1/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
new file mode 100644
index 0000000..30a5a41
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/LogHandlerTestRun.java
@@ -0,0 +1,82 @@
+/*
+ * 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.twill.yarn;
+
+import org.apache.twill.api.AbstractTwillRunnable;
+import org.apache.twill.api.TwillController;
+import org.apache.twill.api.TwillRunner;
+import org.apache.twill.api.logging.LogEntry;
+import org.apache.twill.api.logging.LogHandler;
+import org.apache.twill.common.Services;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Test for LogHandler able to receive logs from AM and runnable.
+ */
+public class LogHandlerTestRun extends BaseYarnTest {
+
+  @Test
+  public void testLogHandler() throws ExecutionException, InterruptedException {
+    final CountDownLatch latch = new CountDownLatch(2);
+
+    LogHandler logHandler = new LogHandler() {
+      @Override
+      public void onLog(LogEntry logEntry) {
+        // Would expect logs from AM and the runnable.
+        if (logEntry.getMessage().startsWith("Starting runnable " + LogRunnable.class.getSimpleName())) {
+          latch.countDown();
+        } else if (logEntry.getMessage().equals("Running")) {
+          latch.countDown();
+        }
+      }
+    };
+
+    TwillRunner runner = YarnTestUtils.getTwillRunner();
+    TwillController controller = runner.prepare(new LogRunnable())
+                                       .addLogHandler(logHandler)
+                                       .start();
+
+    Services.getCompletionFuture(controller).get();
+    latch.await(1, TimeUnit.SECONDS);
+  }
+
+  /**
+   * TwillRunnable for the test case to simply emit one log line.
+   */
+  public static final class LogRunnable extends AbstractTwillRunnable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(LogRunnable.class);
+
+
+    @Override
+    public void run() {
+      LOG.info("Running");
+    }
+
+    @Override
+    public void stop() {
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/146740b1/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
index 912b713..bed613a 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
@@ -31,7 +31,8 @@ import org.junit.runners.Suite;
                       DistributeShellTestRun.class,
                       LocalFileTestRun.class,
                       FailureRestartTestRun.class,
-                      ProvisionTimeoutTestRun.class
+                      ProvisionTimeoutTestRun.class,
+                      LogHandlerTestRun.class
                     })
 public final class YarnTestSuite {
 


[10/50] [abbrv] git commit: Alter version naming to fit incubator release requirement.

Posted by ch...@apache.org.
Alter version naming to fit incubator release requirement.

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

Branch: refs/heads/site
Commit: 61a602a0ad88968ea70e6123b92eb25a34cc3c8d
Parents: cb09429
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Jan 27 14:14:45 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Jan 27 14:14:45 2014 -0800

----------------------------------------------------------------------
 pom.xml                      | 2 +-
 twill-api/pom.xml            | 2 +-
 twill-common/pom.xml         | 2 +-
 twill-core/pom.xml           | 2 +-
 twill-discovery-api/pom.xml  | 2 +-
 twill-discovery-core/pom.xml | 2 +-
 twill-yarn/pom.xml           | 2 +-
 twill-zookeeper/pom.xml      | 2 +-
 8 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/61a602a0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3d7cebe..217a0ce 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
 
     <groupId>org.apache.twill</groupId>
     <artifactId>twill-parent</artifactId>
-    <version>0.1.0-SNAPSHOT</version>
+    <version>0.0.1-incubating-SNAPSHOT</version>
     <packaging>pom</packaging>
     <name>Apache Twill</name>
     <url>http://twill.incubator.apache.org</url>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/61a602a0/twill-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-api/pom.xml b/twill-api/pom.xml
index 49b92c8..29288b4 100644
--- a/twill-api/pom.xml
+++ b/twill-api/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.twill</groupId>
         <artifactId>twill-parent</artifactId>
-        <version>0.1.0-SNAPSHOT</version>
+        <version>0.0.1-incubating-SNAPSHOT</version>
     </parent>
 
     <artifactId>twill-api</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/61a602a0/twill-common/pom.xml
----------------------------------------------------------------------
diff --git a/twill-common/pom.xml b/twill-common/pom.xml
index a4372f6..1d86ed1 100644
--- a/twill-common/pom.xml
+++ b/twill-common/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-SNAPSHOT</version>
+        <version>0.0.1-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/61a602a0/twill-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-core/pom.xml b/twill-core/pom.xml
index d998b40..ae38146 100644
--- a/twill-core/pom.xml
+++ b/twill-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-SNAPSHOT</version>
+        <version>0.0.1-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/61a602a0/twill-discovery-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-api/pom.xml b/twill-discovery-api/pom.xml
index e41b214..296bdde 100644
--- a/twill-discovery-api/pom.xml
+++ b/twill-discovery-api/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-SNAPSHOT</version>
+        <version>0.0.1-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/61a602a0/twill-discovery-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-core/pom.xml b/twill-discovery-core/pom.xml
index 2612138..4f89bfe 100644
--- a/twill-discovery-core/pom.xml
+++ b/twill-discovery-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-SNAPSHOT</version>
+        <version>0.0.1-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/61a602a0/twill-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-yarn/pom.xml b/twill-yarn/pom.xml
index b11bc7a..6a63db7 100644
--- a/twill-yarn/pom.xml
+++ b/twill-yarn/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-SNAPSHOT</version>
+        <version>0.0.1-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/61a602a0/twill-zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/twill-zookeeper/pom.xml b/twill-zookeeper/pom.xml
index e76ee50..6c917d5 100644
--- a/twill-zookeeper/pom.xml
+++ b/twill-zookeeper/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-SNAPSHOT</version>
+        <version>0.0.1-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 


[22/50] [abbrv] git commit: Add license statement.

Posted by ch...@apache.org.
Add license statement.

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

Branch: refs/heads/site
Commit: 5e89ec287a6c3bbb1a459e5536fe3c47cc172f95
Parents: d9e8ead
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Feb 10 13:51:15 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Feb 10 13:51:15 2014 -0800

----------------------------------------------------------------------
 checkstyle.xml | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/5e89ec28/checkstyle.xml
----------------------------------------------------------------------
diff --git a/checkstyle.xml b/checkstyle.xml
index 5fcce66..6404c8f 100644
--- a/checkstyle.xml
+++ b/checkstyle.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
 <!DOCTYPE module PUBLIC
     "-//Puppy Crawl//DTD Check Configuration 1.3//EN"
     "http://www.puppycrawl.com/dtds/configuration_1_3.dtd">


[13/50] [abbrv] git commit: Move Cloudera repo into the cdh profile only.

Posted by ch...@apache.org.
Move Cloudera repo into the cdh profile only.

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

Branch: refs/heads/site
Commit: 95b6c64bf544323bc66331e45492646bdbd36fac
Parents: 098a0ca
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Jan 27 17:09:32 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Jan 27 17:09:32 2014 -0800

----------------------------------------------------------------------
 pom.xml | 25 ++++++++++++-------------
 1 file changed, 12 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/95b6c64b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index dbebdae..a95ea6f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -166,19 +166,6 @@
         </mailingList>
     </mailingLists>
 
-    <repositories>
-        <repository>
-            <id>cloudera-releases</id>
-            <url>https://repository.cloudera.com/artifactory/cloudera-repos</url>
-            <releases>
-                <enabled>true</enabled>
-            </releases>
-            <snapshots>
-                <enabled>false</enabled>
-            </snapshots>
-        </repository>
-    </repositories>
-
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <surefire.redirectTestOutputToFile>true</surefire.redirectTestOutputToFile>
@@ -380,6 +367,18 @@
             <properties>
                 <hadoop.version>2.0.0-cdh4.4.0</hadoop.version>
             </properties>
+            <repositories>
+                <repository>
+                    <id>cloudera-releases</id>
+                    <url>https://repository.cloudera.com/artifactory/cloudera-repos</url>
+                    <releases>
+                        <enabled>true</enabled>
+                    </releases>
+                    <snapshots>
+                        <enabled>false</enabled>
+                    </snapshots>
+                </repository>
+            </repositories>
             <build>
                 <plugins>
                     <plugin>


[33/50] [abbrv] git commit: exposes the YARN client ContainerRequest hosts and racks options into ResourceSpecification #TWILL-40

Posted by ch...@apache.org.
exposes the YARN client ContainerRequest hosts and racks options into ResourceSpecification #TWILL-40

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: ac67f1f620099e86b5503ff847862306e16c4db9
Parents: a77b67c
Author: Fabian Murariu <mu...@gmail.com>
Authored: Thu Feb 13 19:01:03 2014 +0200
Committer: Terence Yim <te...@continuuity.com>
Committed: Fri Feb 21 12:33:26 2014 -0800

----------------------------------------------------------------------
 pom.xml                                         |  6 +++
 .../apache/twill/api/ResourceSpecification.java | 53 +++++++++++++++-----
 .../internal/DefaultResourceSpecification.java  | 18 +++++++
 twill-core/pom.xml                              |  4 ++
 .../json/ResourceSpecificationCodec.java        | 21 ++++----
 .../json/ResourceSpecificationCodecTest.java    | 51 +++++++++++++++++++
 .../appmaster/ApplicationMasterService.java     |  5 +-
 .../twill/internal/yarn/YarnAMClient.java       | 21 ++++----
 .../apache/twill/yarn/EchoServerTestRun.java    |  2 +
 9 files changed, 147 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/ac67f1f6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2141000..0e692f1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -797,6 +797,12 @@
                 <scope>test</scope>
             </dependency>
             <dependency>
+                <groupId>org.unitils</groupId>
+                <artifactId>unitils-core</artifactId>
+                <version>3.3</version>
+                <scope>test</scope>
+            </dependency>
+            <dependency>
                 <groupId>org.apache.commons</groupId>
                 <artifactId>commons-compress</artifactId>
                 <version>${commons-compress.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/ac67f1f6/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
index 0bab811..75427a7 100644
--- a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
@@ -79,6 +79,20 @@ public interface ResourceSpecification {
   int getInstances();
 
   /**
+   * Returns the execution hosts, expects Fully Qualified Domain Names host + domain
+   * This is a suggestion for the scheduler depending on cluster load it may ignore it
+   * @return An array containing the hosts where the containers should run
+   */
+  String[] getHosts();
+
+  /**
+   * Returns the execution racks
+   * This is a suggestion for the scheduler depending on cluster load it may ignore it
+   * @return An array containing the racks where the containers should run
+   */
+  String[] getRacks();
+
+  /**
    * Builder for creating {@link ResourceSpecification}.
    */
   static final class Builder {
@@ -88,11 +102,31 @@ public interface ResourceSpecification {
     private int uplink = -1;
     private int downlink = -1;
     private int instances = 1;
+    private String[] hosts = new String[0];
+    private String[] racks = new String[0];
 
     public static CoreSetter with() {
       return new Builder().new CoreSetter();
     }
 
+    public final class HostsSetter extends Build {
+      public RackSetter setHosts(String... hosts) {
+        if (hosts != null) {
+          Builder.this.hosts = hosts.clone();
+        }
+        return new RackSetter();
+      }
+    }
+
+    public final class RackSetter extends Build {
+      public AfterRacks setRacks(String... racks) {
+        if (racks != null) {
+          Builder.this.racks = racks.clone();
+        }
+        return new AfterRacks();
+      }
+    }
+
     public final class CoreSetter {
       @Deprecated
       public MemorySetter setCores(int cores) {
@@ -114,13 +148,13 @@ public interface ResourceSpecification {
     }
 
     public final class AfterMemory extends Build {
-      public AfterInstances setInstances(int instances) {
+      public HostsSetter setInstances(int instances) {
         Builder.this.instances = instances;
-        return new AfterInstances();
+        return new HostsSetter();
       }
     }
 
-    public final class AfterInstances extends Build {
+    public final class AfterRacks extends Build {
       public AfterUplink setUplink(int uplink, SizeUnit unit) {
         Builder.this.uplink = uplink * unit.multiplier;
         return new AfterUplink();
@@ -128,23 +162,18 @@ public interface ResourceSpecification {
     }
 
     public final class AfterUplink extends Build {
-      public AfterDownlink setDownlink(int downlink, SizeUnit unit) {
+      public Done setDownlink(int downlink, SizeUnit unit) {
         Builder.this.downlink = downlink * unit.multiplier;
-        return new AfterDownlink();
+        return new Done();
       }
     }
 
-    public final class AfterDownlink extends Build {
-
-      @Override
-      public ResourceSpecification build() {
-        return super.build();
-      }
+    public final class Done extends Build {
     }
 
     public abstract class Build {
       public ResourceSpecification build() {
-        return new DefaultResourceSpecification(cores, memory, instances, uplink, downlink);
+        return new DefaultResourceSpecification(cores, memory, instances, uplink, downlink, hosts, racks);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/ac67f1f6/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java b/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
index 1327ce5..4451ac1 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
@@ -28,13 +28,21 @@ public final class DefaultResourceSpecification implements ResourceSpecification
   private final int instances;
   private final int uplink;
   private final int downlink;
+  private final String[] hosts;
+  private final String[] racks;
 
   public DefaultResourceSpecification(int virtualCores, int memorySize, int instances, int uplink, int downlink) {
+    this(virtualCores, memorySize, instances, uplink, downlink, new String[0], new String[0]);
+  }
+
+  public DefaultResourceSpecification(int virtualCores, int memorySize, int instances, int uplink, int downlink, String[] hosts, String[] racks) {
     this.virtualCores = virtualCores;
     this.memorySize = memorySize;
     this.instances = instances;
     this.uplink = uplink;
     this.downlink = downlink;
+    this.hosts = hosts;
+    this.racks = racks;
   }
 
   @Deprecated
@@ -59,6 +67,16 @@ public final class DefaultResourceSpecification implements ResourceSpecification
   }
 
   @Override
+  public String[] getHosts() {
+    return hosts.clone();
+  }
+
+  @Override
+  public String[] getRacks() {
+    return racks.clone();
+  }
+
+  @Override
   public int getUplink() {
     return uplink;
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/ac67f1f6/twill-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-core/pom.xml b/twill-core/pom.xml
index 859c653..124547c 100644
--- a/twill-core/pom.xml
+++ b/twill-core/pom.xml
@@ -86,6 +86,10 @@
             <artifactId>junit</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.unitils</groupId>
+            <artifactId>unitils-core</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.commons</groupId>
             <artifactId>commons-compress</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/ac67f1f6/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
index d3b9707..e88045c 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
@@ -17,13 +17,7 @@
  */
 package org.apache.twill.internal.json;
 
-import com.google.gson.JsonDeserializationContext;
-import com.google.gson.JsonDeserializer;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonObject;
-import com.google.gson.JsonParseException;
-import com.google.gson.JsonSerializationContext;
-import com.google.gson.JsonSerializer;
+import com.google.gson.*;
 import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.internal.DefaultResourceSpecification;
 
@@ -38,13 +32,13 @@ final class ResourceSpecificationCodec implements JsonSerializer<ResourceSpecifi
   @Override
   public JsonElement serialize(ResourceSpecification src, Type typeOfSrc, JsonSerializationContext context) {
     JsonObject json = new JsonObject();
-
     json.addProperty("cores", src.getVirtualCores());
     json.addProperty("memorySize", src.getMemorySize());
     json.addProperty("instances", src.getInstances());
     json.addProperty("uplink", src.getUplink());
     json.addProperty("downlink", src.getDownlink());
-
+    json.add("hosts", context.serialize(src.getHosts()));
+    json.add("racks", context.serialize(src.getRacks()));
     return json;
   }
 
@@ -52,10 +46,17 @@ final class ResourceSpecificationCodec implements JsonSerializer<ResourceSpecifi
   public ResourceSpecification deserialize(JsonElement json, Type typeOfT,
                                            JsonDeserializationContext context) throws JsonParseException {
     JsonObject jsonObj = json.getAsJsonObject();
+    final String[] hosts = context.deserialize(jsonObj.getAsJsonArray("hosts"), String[].class);
+    final String[] racks = context.deserialize(jsonObj.getAsJsonArray("racks"), String[].class);
     return new DefaultResourceSpecification(jsonObj.get("cores").getAsInt(),
                                             jsonObj.get("memorySize").getAsInt(),
                                             jsonObj.get("instances").getAsInt(),
                                             jsonObj.get("uplink").getAsInt(),
-                                            jsonObj.get("downlink").getAsInt());
+                                            jsonObj.get("downlink").getAsInt(),
+                                            hosts,
+                                            racks);
+
+
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/ac67f1f6/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java b/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
new file mode 100644
index 0000000..7bd4f39
--- /dev/null
+++ b/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
@@ -0,0 +1,51 @@
+package org.apache.twill.internal.json;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonElement;
+import org.apache.twill.api.ResourceSpecification;
+import org.apache.twill.internal.DefaultResourceSpecification;
+import org.junit.Test;
+
+import static org.apache.twill.api.ResourceSpecification.SizeUnit.GIGA;
+import static org.junit.Assert.assertEquals;
+import static org.unitils.reflectionassert.ReflectionAssert.assertLenientEquals;
+
+public class ResourceSpecificationCodecTest {
+  private final Gson gson = new GsonBuilder().serializeNulls()
+          .registerTypeAdapter(ResourceSpecification.class, new ResourceSpecificationCodec())
+          .registerTypeAdapter(DefaultResourceSpecification.class, new ResourceSpecificationCodec())
+          .create();
+
+  @Test
+  public void testCodec() throws Exception {
+    String expectedString = "{\"cores\":2,\"memorySize\":1024,\"instances\":2,\"uplink\":100,\"downlink\":100,\"hosts\":[\"one1\",\"two2\"],\"racks\":[\"three3\"]}";
+    final ResourceSpecification expected =
+            new DefaultResourceSpecification(2, 1024, 2, 100, 100, new String[]{"one1", "two2"}, new String[]{"three3"});
+    final String actualString = gson.toJson(expected);
+    assertEquals(expectedString, actualString);
+
+    final JsonElement expectedJson = gson.toJsonTree(expected);
+    final ResourceSpecification actual = gson.fromJson(expectedJson, DefaultResourceSpecification.class);
+    final JsonElement actualJson = gson.toJsonTree(actual);
+
+    assertEquals(expectedJson, actualJson);
+    assertLenientEquals(expected, actual);
+  }
+
+  @Test
+  public void testBuilder() throws Exception {
+    final ResourceSpecification actual = ResourceSpecification.Builder.with()
+            .setVirtualCores(5)
+            .setMemory(4, GIGA)
+            .setInstances(3)
+            .setHosts("a1", "b2", "c3")
+            .setRacks("r2")
+            .setUplink(10, GIGA)
+            .setDownlink(5, GIGA).build();
+    final DefaultResourceSpecification expectd =
+            new DefaultResourceSpecification(5, 4096, 3, 10240, 5120, new String[]{"a1", "b2", "c3"}, new String[]{"r2"});
+    assertLenientEquals(expectd, actual);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/ac67f1f6/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
index 7caedad..3a6ce20 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/ApplicationMasterService.java
@@ -507,7 +507,10 @@ public final class ApplicationMasterService extends AbstractTwillService {
       if (newContainers > 0) {
         // TODO: Allow user to set priority?
         LOG.info("Request {} container with capability {}", newContainers, capability);
-        String requestId = amClient.addContainerRequest(capability, newContainers).setPriority(0).apply();
+        String requestId = amClient.addContainerRequest(capability, newContainers)
+                .addHosts(runtimeSpec.getResourceSpecification().getHosts())
+                .addRacks(runtimeSpec.getResourceSpecification().getRacks())
+                .setPriority(0).apply();
         provisioning.add(new ProvisionRequest(runtimeSpec, requestId, newContainers));
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/ac67f1f6/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
index b0dbce0..c8da649 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
@@ -27,10 +27,9 @@ import org.apache.twill.internal.ProcessLauncher;
 
 import java.net.InetSocketAddress;
 import java.net.URL;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
+import java.util.*;
+
+import static org.apache.commons.lang.ArrayUtils.isEmpty;
 
 /**
  * This interface provides abstraction for AM to interacts with YARN to abstract out YARN version specific
@@ -54,12 +53,12 @@ public interface YarnAMClient extends Service {
       this.count = count;
     }
 
-    public ContainerRequestBuilder addHosts(String firstHost, String...moreHosts) {
-      return add(hosts, firstHost, moreHosts);
+    public ContainerRequestBuilder addHosts(String...newHosts) {
+      return add(hosts, newHosts);
     }
 
-    public ContainerRequestBuilder addRacks(String firstRack, String...moreRacks) {
-      return add(racks, firstRack, moreRacks);
+    public ContainerRequestBuilder addRacks(String...newRacks) {
+      return add(racks, newRacks);
     }
 
     public ContainerRequestBuilder setPriority(int prio) {
@@ -72,9 +71,9 @@ public interface YarnAMClient extends Service {
      */
     public abstract String apply();
 
-    private <T> ContainerRequestBuilder add(Collection<T> collection, T first, T... more) {
-      collection.add(first);
-      Collections.addAll(collection, more);
+    private <T> ContainerRequestBuilder add(Collection<T> collection, T... more) {
+      if (!isEmpty(more))
+        Collections.addAll(collection, more);
       return this;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/ac67f1f6/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
index 23fc82b..1c3034c 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
@@ -61,6 +61,8 @@ public final class EchoServerTestRun extends BaseYarnTest {
                                                          .setVirtualCores(1)
                                                          .setMemory(1, ResourceSpecification.SizeUnit.GIGA)
                                                          .setInstances(2)
+                                                         .setHosts("someHost1.domain.no","someHost2.domain.no") /*demo only ignored in this test*/
+                                                         .setRacks("someRack1") /* demo only ignored in this test*/
                                                          .build())
                                         .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
                                         .withApplicationArguments("echo")


[49/50] [abbrv] git commit: (TWILL-74) Added authentication and ACL support to ZKClient.

Posted by ch...@apache.org.
(TWILL-74) Added authentication and ACL support to ZKClient.

* New method, addAuthInfo, added to ZkClientService.Builder for creating ZKClient with authentication.
* New methods (create, getACL, setACL) to support ACL are added to ZKClient.
* Refactor ZKClient hierarchy to have a AbstractZKClient to make sure correct behavior of method delegation for children classes.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 9393df80ba1c0b83a162c250c64ebd812d9db214
Parents: 2c3cf39
Author: Terence Yim <te...@continuuity.com>
Authored: Thu Apr 17 18:40:36 2014 -0700
Committer: Terence Yim <te...@continuuity.com>
Committed: Fri Apr 18 13:36:41 2014 -0700

----------------------------------------------------------------------
 .../twill/internal/zookeeper/BasicACLData.java  |  48 ++++++
 .../zookeeper/DefaultZKClientService.java       | 145 ++++++++++++-------
 .../zookeeper/FailureRetryZKClient.java         |  72 ++++-----
 .../internal/zookeeper/NamespaceZKClient.java   |  44 ++----
 .../zookeeper/RewatchOnExpireZKClient.java      |   6 +-
 .../org/apache/twill/zookeeper/ACLData.java     |  39 +++++
 .../twill/zookeeper/AbstractZKClient.java       |  80 ++++++++++
 .../twill/zookeeper/ForwardingZKClient.java     |  45 ++----
 .../org/apache/twill/zookeeper/NodeData.java    |   2 +-
 .../apache/twill/zookeeper/RetryStrategy.java   |   4 +-
 .../org/apache/twill/zookeeper/ZKClient.java    |  58 +++++++-
 .../apache/twill/zookeeper/ZKClientService.java |  19 ++-
 .../twill/zookeeper/ZKClientServices.java       |  46 ++----
 .../apache/twill/zookeeper/ZKClientTest.java    |  75 +++++++++-
 14 files changed, 480 insertions(+), 203 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicACLData.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicACLData.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicACLData.java
new file mode 100644
index 0000000..d31de8b
--- /dev/null
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/BasicACLData.java
@@ -0,0 +1,48 @@
+/*
+ * 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.twill.internal.zookeeper;
+
+import org.apache.twill.zookeeper.ACLData;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+
+import java.util.List;
+
+/**
+ * A straightforward implementation of {@link ACLData}.
+ */
+final class BasicACLData implements ACLData {
+
+  private final List<ACL> acl;
+  private final Stat stat;
+
+  BasicACLData(List<ACL> acl, Stat stat) {
+    this.acl = acl;
+    this.stat = stat;
+  }
+
+  @Override
+  public List<ACL> getACL() {
+    return acl;
+  }
+
+  @Override
+  public Stat getStat() {
+    return stat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/DefaultZKClientService.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/DefaultZKClientService.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/DefaultZKClientService.java
index 84b3a8d..7c9bd08 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/DefaultZKClientService.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/DefaultZKClientService.java
@@ -17,15 +17,20 @@
  */
 package org.apache.twill.internal.zookeeper;
 
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Multimap;
 import com.google.common.util.concurrent.AbstractService;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Service;
 import org.apache.twill.common.Threads;
+import org.apache.twill.zookeeper.ACLData;
+import org.apache.twill.zookeeper.AbstractZKClient;
 import org.apache.twill.zookeeper.NodeChildren;
 import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.OperationFuture;
@@ -43,7 +48,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
@@ -56,33 +63,37 @@ import javax.annotation.Nullable;
 /**
  * The base implementation of {@link ZKClientService}.
  */
-public final class DefaultZKClientService implements ZKClientService {
+public final class DefaultZKClientService extends AbstractZKClient implements ZKClientService {
 
   private static final Logger LOG = LoggerFactory.getLogger(DefaultZKClientService.class);
 
   private final String zkStr;
   private final int sessionTimeout;
   private final List<Watcher> connectionWatchers;
+  private final Multimap<String, byte[]> authInfos;
   private final AtomicReference<ZooKeeper> zooKeeper;
-  private final Function<String, List<ACL>> aclMapper;
   private final Service serviceDelegate;
   private ExecutorService eventExecutor;
 
+  /**
+   * Creates a new instance.
+   * @deprecated Use {@link ZKClientService.Builder} instead.
+   */
+  @Deprecated
+  @SuppressWarnings("unused")
   public DefaultZKClientService(String zkStr, int sessionTimeout, Watcher connectionWatcher) {
+    this(zkStr, sessionTimeout, connectionWatcher, ImmutableMultimap.<String, byte[]>of());
+  }
+
+  public DefaultZKClientService(String zkStr, int sessionTimeout,
+                                Watcher connectionWatcher, Multimap<String, byte[]> authInfos) {
     this.zkStr = zkStr;
     this.sessionTimeout = sessionTimeout;
     this.connectionWatchers = new CopyOnWriteArrayList<Watcher>();
+    this.authInfos = copyAuthInfo(authInfos);
     addConnectionWatcher(connectionWatcher);
 
     this.zooKeeper = new AtomicReference<ZooKeeper>();
-
-    // TODO (terence): Add ACL
-    aclMapper = new Function<String, List<ACL>>() {
-      @Override
-      public List<ACL> apply(String input) {
-        return ZooDefs.Ids.OPEN_ACL_UNSAFE;
-      }
-    };
     serviceDelegate = new ServiceDelegate();
   }
 
@@ -105,23 +116,19 @@ public final class DefaultZKClientService implements ZKClientService {
   }
 
   @Override
-  public OperationFuture<String> create(String path, byte[] data, CreateMode createMode) {
-    return create(path, data, createMode, true);
-  }
-
-  @Override
-  public OperationFuture<String> create(String path, @Nullable byte[] data,
-                                        CreateMode createMode, boolean createParent) {
-    return doCreate(path, data, createMode, createParent, false);
+  public OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode,
+                                        boolean createParent, Iterable<ACL> acl) {
+    return doCreate(path, data, createMode, createParent, ImmutableList.copyOf(acl), false);
   }
 
   private OperationFuture<String> doCreate(final String path,
-                                        @Nullable final byte[] data,
-                                        final CreateMode createMode,
-                                        final boolean createParent,
-                                        final boolean ignoreNodeExists) {
+                                           @Nullable final byte[] data,
+                                           final CreateMode createMode,
+                                           final boolean createParent,
+                                           final List<ACL> acl,
+                                           final boolean ignoreNodeExists) {
     final SettableOperationFuture<String> createFuture = SettableOperationFuture.create(path, eventExecutor);
-    getZooKeeper().create(path, data, aclMapper.apply(path), createMode, Callbacks.STRING, createFuture);
+    getZooKeeper().create(path, data, acl, createMode, Callbacks.STRING, createFuture);
     if (!createParent) {
       return createFuture;
     }
@@ -148,14 +155,14 @@ public final class DefaultZKClientService implements ZKClientService {
           result.setException(t);
           return;
         }
-        // Watch for parent creation complete
-        Futures.addCallback(
-          doCreate(parentPath, null, CreateMode.PERSISTENT, createParent, true), new FutureCallback<String>() {
+        // Watch for parent creation complete. Parent is created with the unsafe ACL.
+        Futures.addCallback(doCreate(parentPath, null, CreateMode.PERSISTENT,
+                                     true, ZooDefs.Ids.OPEN_ACL_UNSAFE, true), new FutureCallback<String>() {
           @Override
           public void onSuccess(String parentPath) {
             // Create the requested path again
             Futures.addCallback(
-              doCreate(path, data, createMode, false, ignoreNodeExists), new FutureCallback<String>() {
+              doCreate(path, data, createMode, false, acl, ignoreNodeExists), new FutureCallback<String>() {
               @Override
               public void onSuccess(String pathResult) {
                 result.set(pathResult);
@@ -219,11 +226,6 @@ public final class DefaultZKClientService implements ZKClientService {
   }
 
   @Override
-  public OperationFuture<Stat> exists(String path) {
-    return exists(path, null);
-  }
-
-  @Override
   public OperationFuture<Stat> exists(String path, Watcher watcher) {
     SettableOperationFuture<Stat> result = SettableOperationFuture.create(path, eventExecutor);
     getZooKeeper().exists(path, wrapWatcher(watcher), Callbacks.STAT_NONODE, result);
@@ -231,11 +233,6 @@ public final class DefaultZKClientService implements ZKClientService {
   }
 
   @Override
-  public OperationFuture<NodeChildren> getChildren(String path) {
-    return getChildren(path, null);
-  }
-
-  @Override
   public OperationFuture<NodeChildren> getChildren(String path, Watcher watcher) {
     SettableOperationFuture<NodeChildren> result = SettableOperationFuture.create(path, eventExecutor);
     getZooKeeper().getChildren(path, wrapWatcher(watcher), Callbacks.CHILDREN, result);
@@ -243,11 +240,6 @@ public final class DefaultZKClientService implements ZKClientService {
   }
 
   @Override
-  public OperationFuture<NodeData> getData(String path) {
-    return getData(path, null);
-  }
-
-  @Override
   public OperationFuture<NodeData> getData(String path, Watcher watcher) {
     SettableOperationFuture<NodeData> result = SettableOperationFuture.create(path, eventExecutor);
     getZooKeeper().getData(path, wrapWatcher(watcher), Callbacks.DATA, result);
@@ -256,11 +248,6 @@ public final class DefaultZKClientService implements ZKClientService {
   }
 
   @Override
-  public OperationFuture<Stat> setData(String path, byte[] data) {
-    return setData(path, data, -1);
-  }
-
-  @Override
   public OperationFuture<Stat> setData(String dataPath, byte[] data, int version) {
     SettableOperationFuture<Stat> result = SettableOperationFuture.create(dataPath, eventExecutor);
     getZooKeeper().setData(dataPath, data, version, Callbacks.STAT, result);
@@ -268,11 +255,6 @@ public final class DefaultZKClientService implements ZKClientService {
   }
 
   @Override
-  public OperationFuture<String> delete(String path) {
-    return delete(path, -1);
-  }
-
-  @Override
   public OperationFuture<String> delete(String deletePath, int version) {
     SettableOperationFuture<String> result = SettableOperationFuture.create(deletePath, eventExecutor);
     getZooKeeper().delete(deletePath, version, Callbacks.VOID, result);
@@ -280,6 +262,20 @@ public final class DefaultZKClientService implements ZKClientService {
   }
 
   @Override
+  public OperationFuture<ACLData> getACL(String path) {
+    SettableOperationFuture<ACLData> result = SettableOperationFuture.create(path, eventExecutor);
+    getZooKeeper().getACL(path, new Stat(), Callbacks.ACL, result);
+    return result;
+  }
+
+  @Override
+  public OperationFuture<Stat> setACL(String path, Iterable<ACL> acl, int version) {
+    SettableOperationFuture<Stat> result = SettableOperationFuture.create(path, eventExecutor);
+    getZooKeeper().setACL(path, ImmutableList.copyOf(acl), version, Callbacks.STAT, result);
+    return result;
+  }
+
+  @Override
   public Supplier<ZooKeeper> getZooKeeperSupplier() {
     return new Supplier<ZooKeeper>() {
       @Override
@@ -359,6 +355,20 @@ public final class DefaultZKClientService implements ZKClientService {
     };
   }
 
+  /**
+   * Creates a deep copy of the given authInfos multimap.
+   */
+  private Multimap<String, byte[]> copyAuthInfo(Multimap<String, byte[]> authInfos) {
+    Multimap<String, byte[]> result = ArrayListMultimap.create();
+
+    for (Map.Entry<String, byte[]> entry : authInfos.entries()) {
+      byte[] info = entry.getValue();
+      result.put(entry.getKey(), info == null ? null : Arrays.copyOf(info, info.length));
+    }
+
+    return result;
+  }
+
   private final class ServiceDelegate extends AbstractService implements Watcher {
 
     @Override
@@ -374,7 +384,7 @@ public final class DefaultZKClientService implements ZKClientService {
       };
 
       try {
-        zooKeeper.set(new ZooKeeper(zkStr, sessionTimeout, this));
+        zooKeeper.set(createZooKeeper());
       } catch (IOException e) {
         notifyFailed(e);
       }
@@ -410,7 +420,7 @@ public final class DefaultZKClientService implements ZKClientService {
             @Override
             public void run() {
               try {
-                zooKeeper.set(new ZooKeeper(zkStr, sessionTimeout, ServiceDelegate.this));
+                zooKeeper.set(createZooKeeper());
               } catch (IOException e) {
                 zooKeeper.set(null);
                 notifyFailed(e);
@@ -428,6 +438,17 @@ public final class DefaultZKClientService implements ZKClientService {
         }
       }
     }
+
+    /**
+     * Creates a new ZooKeeper connection.
+     */
+    private ZooKeeper createZooKeeper() throws IOException {
+      ZooKeeper zk = new ZooKeeper(zkStr, sessionTimeout, this);
+      for (Map.Entry<String, byte[]> authInfo : authInfos.entries()) {
+        zk.addAuthInfo(authInfo.getKey(), authInfo.getValue());
+      }
+      return zk;
+    }
   }
 
   /**
@@ -521,5 +542,19 @@ public final class DefaultZKClientService implements ZKClientService {
         result.setException(KeeperException.create(code, result.getRequestPath()));
       }
     };
+
+    static final AsyncCallback.ACLCallback ACL = new AsyncCallback.ACLCallback() {
+      @Override
+      @SuppressWarnings("unchecked")
+      public void processResult(int rc, String path, Object ctx, List<ACL> acl, Stat stat) {
+        SettableOperationFuture<ACLData> result = (SettableOperationFuture<ACLData>) ctx;
+        KeeperException.Code code = KeeperException.Code.get(rc);
+        if (code == KeeperException.Code.OK) {
+          result.set(new BasicACLData(acl, stat));
+          return;
+        }
+        result.setException(KeeperException.create(code, result.getRequestPath()));
+      }
+    };
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/FailureRetryZKClient.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/FailureRetryZKClient.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/FailureRetryZKClient.java
index aa11730..73ee308 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/FailureRetryZKClient.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/FailureRetryZKClient.java
@@ -21,6 +21,7 @@ import com.google.common.base.Supplier;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import org.apache.twill.common.Threads;
+import org.apache.twill.zookeeper.ACLData;
 import org.apache.twill.zookeeper.ForwardingZKClient;
 import org.apache.twill.zookeeper.NodeChildren;
 import org.apache.twill.zookeeper.NodeData;
@@ -30,12 +31,14 @@ import org.apache.twill.zookeeper.RetryStrategy.OperationType;
 import org.apache.twill.zookeeper.ZKClient;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import javax.annotation.Nullable;
 
 /**
  * A {@link ZKClient} that will invoke {@link RetryStrategy} on operation failure.
@@ -56,37 +59,26 @@ public final class FailureRetryZKClient extends ForwardingZKClient {
   }
 
   @Override
-  public OperationFuture<String> create(String path, byte[] data, CreateMode createMode) {
-    return create(path, data, createMode, true);
-  }
-
-  @Override
-  public OperationFuture<String> create(final String path, final byte[] data,
-                                        final CreateMode createMode, final boolean createParent) {
-
+  public OperationFuture<String> create(final String path, @Nullable final byte[] data, final CreateMode createMode,
+                                        final boolean createParent, final Iterable<ACL> acl) {
     // No retry for any SEQUENTIAL node, as some algorithms depends on only one sequential node being created.
     if (createMode == CreateMode.PERSISTENT_SEQUENTIAL || createMode == CreateMode.EPHEMERAL_SEQUENTIAL) {
-      return super.create(path, data, createMode, createParent);
+      return super.create(path, data, createMode, createParent, acl);
     }
 
     final SettableOperationFuture<String> result = SettableOperationFuture.create(path, Threads.SAME_THREAD_EXECUTOR);
-    Futures.addCallback(super.create(path, data, createMode, createParent),
+    Futures.addCallback(super.create(path, data, createMode, createParent, acl),
                         new OperationFutureCallback<String>(OperationType.CREATE, System.currentTimeMillis(),
                                                             path, result, new Supplier<OperationFuture<String>>() {
                           @Override
                           public OperationFuture<String> get() {
-                            return FailureRetryZKClient.super.create(path, data, createMode, createParent);
+                            return FailureRetryZKClient.super.create(path, data, createMode, createParent, acl);
                           }
                         }));
     return result;
   }
 
   @Override
-  public OperationFuture<Stat> exists(String path) {
-    return exists(path, null);
-  }
-
-  @Override
   public OperationFuture<Stat> exists(final String path, final Watcher watcher) {
     final SettableOperationFuture<Stat> result = SettableOperationFuture.create(path, Threads.SAME_THREAD_EXECUTOR);
     Futures.addCallback(super.exists(path, watcher),
@@ -101,11 +93,6 @@ public final class FailureRetryZKClient extends ForwardingZKClient {
   }
 
   @Override
-  public OperationFuture<NodeChildren> getChildren(String path) {
-    return getChildren(path, null);
-  }
-
-  @Override
   public OperationFuture<NodeChildren> getChildren(final String path, final Watcher watcher) {
     final SettableOperationFuture<NodeChildren> result = SettableOperationFuture.create(path,
                                                                                         Threads.SAME_THREAD_EXECUTOR);
@@ -122,11 +109,6 @@ public final class FailureRetryZKClient extends ForwardingZKClient {
   }
 
   @Override
-  public OperationFuture<NodeData> getData(String path) {
-    return getData(path, null);
-  }
-
-  @Override
   public OperationFuture<NodeData> getData(final String path, final Watcher watcher) {
     final SettableOperationFuture<NodeData> result = SettableOperationFuture.create(path, Threads.SAME_THREAD_EXECUTOR);
     Futures.addCallback(super.getData(path, watcher),
@@ -141,11 +123,6 @@ public final class FailureRetryZKClient extends ForwardingZKClient {
   }
 
   @Override
-  public OperationFuture<Stat> setData(String path, byte[] data) {
-    return setData(path, data, -1);
-  }
-
-  @Override
   public OperationFuture<Stat> setData(final String dataPath, final byte[] data, final int version) {
     final SettableOperationFuture<Stat> result = SettableOperationFuture.create(dataPath, Threads.SAME_THREAD_EXECUTOR);
     Futures.addCallback(super.setData(dataPath, data, version),
@@ -160,11 +137,6 @@ public final class FailureRetryZKClient extends ForwardingZKClient {
   }
 
   @Override
-  public OperationFuture<String> delete(String path) {
-    return delete(path, -1);
-  }
-
-  @Override
   public OperationFuture<String> delete(final String deletePath, final int version) {
     final SettableOperationFuture<String> result = SettableOperationFuture.create(deletePath,
                                                                                   Threads.SAME_THREAD_EXECUTOR);
@@ -180,6 +152,34 @@ public final class FailureRetryZKClient extends ForwardingZKClient {
     return result;
   }
 
+  @Override
+  public OperationFuture<ACLData> getACL(final String path) {
+    final SettableOperationFuture<ACLData> result = SettableOperationFuture.create(path, Threads.SAME_THREAD_EXECUTOR);
+    Futures.addCallback(super.getACL(path),
+                        new OperationFutureCallback<ACLData>(OperationType.GET_ACL, System.currentTimeMillis(),
+                                                          path, result, new Supplier<OperationFuture<ACLData>>() {
+                          @Override
+                          public OperationFuture<ACLData> get() {
+                            return FailureRetryZKClient.super.getACL(path);
+                          }
+                        }));
+    return result;
+  }
+
+  @Override
+  public OperationFuture<Stat> setACL(final String path, final Iterable<ACL> acl, final int version) {
+    final SettableOperationFuture<Stat> result = SettableOperationFuture.create(path, Threads.SAME_THREAD_EXECUTOR);
+    Futures.addCallback(super.setACL(path, acl, version),
+                        new OperationFutureCallback<Stat>(OperationType.SET_ACL, System.currentTimeMillis(),
+                                                          path, result, new Supplier<OperationFuture<Stat>>() {
+                          @Override
+                          public OperationFuture<Stat> get() {
+                            return FailureRetryZKClient.super.setACL(path, acl, version);
+                          }
+                        }));
+    return result;
+  }
+
   /**
    * Callback to watch for operation result and trigger retry if necessary.
    * @param <V> Type of operation result.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/NamespaceZKClient.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/NamespaceZKClient.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/NamespaceZKClient.java
index 7d3c268..beca5e8 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/NamespaceZKClient.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/NamespaceZKClient.java
@@ -20,6 +20,7 @@ package org.apache.twill.internal.zookeeper;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import org.apache.twill.common.Threads;
+import org.apache.twill.zookeeper.ACLData;
 import org.apache.twill.zookeeper.ForwardingZKClient;
 import org.apache.twill.zookeeper.NodeChildren;
 import org.apache.twill.zookeeper.NodeData;
@@ -27,6 +28,7 @@ import org.apache.twill.zookeeper.OperationFuture;
 import org.apache.twill.zookeeper.ZKClient;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 
 import javax.annotation.Nullable;
@@ -65,65 +67,45 @@ public final class NamespaceZKClient extends ForwardingZKClient {
   }
 
   @Override
-  public OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode) {
-    return relayPath(delegate.create(namespace + path, data, createMode), this.<String>createFuture(path));
-  }
-
-  @Override
-  public OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode,
-                                        boolean createParent) {
-    return relayPath(delegate.create(namespace + path, data, createMode, createParent),
+  public OperationFuture<String> create(String path, @Nullable byte[] data,
+                                        CreateMode createMode, boolean createParent, Iterable<ACL> acl) {
+    return relayPath(delegate.create(namespace + path, data, createMode, createParent, acl),
                      this.<String>createFuture(path));
   }
 
   @Override
-  public OperationFuture<Stat> exists(String path) {
-    return relayFuture(delegate.exists(namespace + path), this.<Stat>createFuture(path));
-  }
-
-  @Override
   public OperationFuture<Stat> exists(String path, @Nullable Watcher watcher) {
     return relayFuture(delegate.exists(namespace + path, watcher), this.<Stat>createFuture(path));
   }
 
   @Override
-  public OperationFuture<NodeChildren> getChildren(String path) {
-    return relayFuture(delegate.getChildren(namespace + path), this.<NodeChildren>createFuture(path));
-  }
-
-  @Override
   public OperationFuture<NodeChildren> getChildren(String path, @Nullable Watcher watcher) {
     return relayFuture(delegate.getChildren(namespace + path, watcher), this.<NodeChildren>createFuture(path));
   }
 
   @Override
-  public OperationFuture<NodeData> getData(String path) {
-    return relayFuture(delegate.getData(namespace + path), this.<NodeData>createFuture(path));
-  }
-
-  @Override
   public OperationFuture<NodeData> getData(String path, @Nullable Watcher watcher) {
     return relayFuture(delegate.getData(namespace + path, watcher), this.<NodeData>createFuture(path));
   }
 
   @Override
-  public OperationFuture<Stat> setData(String path, byte[] data) {
-    return relayFuture(delegate.setData(namespace + path, data), this.<Stat>createFuture(path));
+  public OperationFuture<Stat> setData(String dataPath, byte[] data, int version) {
+    return relayFuture(delegate.setData(namespace + dataPath, data, version), this.<Stat>createFuture(dataPath));
   }
 
   @Override
-  public OperationFuture<Stat> setData(String dataPath, byte[] data, int version) {
-    return relayFuture(delegate.setData(namespace + dataPath, data, version), this.<Stat>createFuture(dataPath));
+  public OperationFuture<String> delete(String deletePath, int version) {
+    return relayPath(delegate.delete(namespace + deletePath, version), this.<String>createFuture(deletePath));
   }
 
   @Override
-  public OperationFuture<String> delete(String path) {
-    return relayPath(delegate.delete(namespace + path), this.<String>createFuture(path));
+  public OperationFuture<ACLData> getACL(String path) {
+    return relayFuture(delegate.getACL(namespace + path), this.<ACLData>createFuture(path));
   }
 
   @Override
-  public OperationFuture<String> delete(String deletePath, int version) {
-    return relayPath(delegate.delete(namespace + deletePath, version), this.<String>createFuture(deletePath));
+  public OperationFuture<Stat> setACL(String path, Iterable<ACL> acl, int version) {
+    return relayFuture(delegate.setACL(namespace + path, acl, version), this.<Stat>createFuture(path));
   }
 
   private <V> SettableOperationFuture<V> createFuture(String path) {

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
index bd3bd2d..ed0e0bd 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
@@ -41,7 +41,7 @@ public final class RewatchOnExpireZKClient extends ForwardingZKClient {
   @Override
   public OperationFuture<Stat> exists(String path, Watcher watcher) {
     if (watcher == null) {
-      return super.exists(path, watcher);
+      return super.exists(path, null);
     }
     final RewatchOnExpireWatcher wrappedWatcher = new RewatchOnExpireWatcher(this, ActionType.EXISTS, path, watcher);
     OperationFuture<Stat> result = super.exists(path, wrappedWatcher);
@@ -62,7 +62,7 @@ public final class RewatchOnExpireZKClient extends ForwardingZKClient {
   @Override
   public OperationFuture<NodeChildren> getChildren(String path, Watcher watcher) {
     if (watcher == null) {
-      return super.getChildren(path, watcher);
+      return super.getChildren(path, null);
     }
     final RewatchOnExpireWatcher wrappedWatcher = new RewatchOnExpireWatcher(this, ActionType.CHILDREN, path, watcher);
     OperationFuture<NodeChildren> result = super.getChildren(path, wrappedWatcher);
@@ -83,7 +83,7 @@ public final class RewatchOnExpireZKClient extends ForwardingZKClient {
   @Override
   public OperationFuture<NodeData> getData(String path, Watcher watcher) {
     if (watcher == null) {
-      return super.getData(path, watcher);
+      return super.getData(path, null);
     }
     final RewatchOnExpireWatcher wrappedWatcher = new RewatchOnExpireWatcher(this, ActionType.DATA, path, watcher);
     OperationFuture<NodeData> result = super.getData(path, wrappedWatcher);

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ACLData.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ACLData.java b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ACLData.java
new file mode 100644
index 0000000..d3a6836
--- /dev/null
+++ b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ACLData.java
@@ -0,0 +1,39 @@
+/*
+ * 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.twill.zookeeper;
+
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+
+import java.util.List;
+
+/**
+ * Represents result of call to {@link ZKClient#getACL(String)}.
+ */
+public interface ACLData {
+
+  /**
+   * @return list of {@link ACL} for the node.
+   */
+  List<ACL> getACL();
+
+  /**
+   * @return The {@link Stat} of the node.
+   */
+  Stat getStat();
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/AbstractZKClient.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/AbstractZKClient.java b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/AbstractZKClient.java
new file mode 100644
index 0000000..208b536
--- /dev/null
+++ b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/AbstractZKClient.java
@@ -0,0 +1,80 @@
+/*
+ * 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.twill.zookeeper;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+
+import javax.annotation.Nullable;
+
+/**
+ * An abstract base implementation of {@link ZKClient} that simplifies implementation by providing forwarding for
+ * methods that are meant to be delegated to other methods.
+ */
+public abstract class AbstractZKClient implements ZKClient {
+
+  @Override
+  public final OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode) {
+    return create(path, data, createMode, true);
+  }
+
+  @Override
+  public final OperationFuture<String> create(String path, @Nullable byte[] data,
+                                        CreateMode createMode, boolean createParent) {
+    return create(path, data, createMode, createParent, ZooDefs.Ids.OPEN_ACL_UNSAFE);
+  }
+
+  @Override
+  public final OperationFuture<String> create(String path, @Nullable byte[] data,
+                                              CreateMode createMode, Iterable<ACL> acl) {
+    return create(path, data, createMode, true, acl);
+  }
+
+  @Override
+  public final OperationFuture<Stat> exists(String path) {
+    return exists(path, null);
+  }
+
+  @Override
+  public final OperationFuture<NodeChildren> getChildren(String path) {
+    return getChildren(path, null);
+  }
+
+  @Override
+  public final OperationFuture<NodeData> getData(String path) {
+    return getData(path, null);
+  }
+
+  @Override
+  public final OperationFuture<Stat> setData(String path, byte[] data) {
+    return setData(path, data, -1);
+  }
+
+  @Override
+  public final OperationFuture<String> delete(String path) {
+    return delete(path, -1);
+  }
+
+  @Override
+  public final OperationFuture<Stat> setACL(String path, Iterable<ACL> acl) {
+    return setACL(path, acl, -1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ForwardingZKClient.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ForwardingZKClient.java b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ForwardingZKClient.java
index 3f3003d..149f7e7 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ForwardingZKClient.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ForwardingZKClient.java
@@ -19,6 +19,7 @@ package org.apache.twill.zookeeper;
 
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 
 import javax.annotation.Nullable;
@@ -26,7 +27,7 @@ import javax.annotation.Nullable;
 /**
  *
  */
-public abstract class ForwardingZKClient implements ZKClient {
+public abstract class ForwardingZKClient extends AbstractZKClient {
 
   private final ZKClient delegate;
 
@@ -54,19 +55,9 @@ public abstract class ForwardingZKClient implements ZKClient {
   }
 
   @Override
-  public OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode) {
-    return create(path, data, createMode, true);
-  }
-
-  @Override
-  public OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode,
-                                        boolean createParent) {
-    return delegate.create(path, data, createMode, createParent);
-  }
-
-  @Override
-  public OperationFuture<Stat> exists(String path) {
-    return exists(path, null);
+  public OperationFuture<String> create(String path, @Nullable byte[] data,
+                                        CreateMode createMode, boolean createParent, Iterable<ACL> acl) {
+    return delegate.create(path, data, createMode, createParent, acl);
   }
 
   @Override
@@ -75,42 +66,32 @@ public abstract class ForwardingZKClient implements ZKClient {
   }
 
   @Override
-  public OperationFuture<NodeChildren> getChildren(String path) {
-    return getChildren(path, null);
-  }
-
-  @Override
   public OperationFuture<NodeChildren> getChildren(String path, @Nullable Watcher watcher) {
     return delegate.getChildren(path, watcher);
   }
 
   @Override
-  public OperationFuture<NodeData> getData(String path) {
-    return getData(path, null);
-  }
-
-  @Override
   public OperationFuture<NodeData> getData(String path, @Nullable Watcher watcher) {
     return delegate.getData(path, watcher);
   }
 
   @Override
-  public OperationFuture<Stat> setData(String path, byte[] data) {
-    return setData(path, data, -1);
+  public OperationFuture<Stat> setData(String dataPath, byte[] data, int version) {
+    return delegate.setData(dataPath, data, version);
   }
 
   @Override
-  public OperationFuture<Stat> setData(String dataPath, byte[] data, int version) {
-    return delegate.setData(dataPath, data, version);
+  public OperationFuture<String> delete(String deletePath, int version) {
+    return delegate.delete(deletePath, version);
   }
 
   @Override
-  public OperationFuture<String> delete(String path) {
-    return delete(path, -1);
+  public OperationFuture<ACLData> getACL(String path) {
+    return delegate.getACL(path);
   }
 
   @Override
-  public OperationFuture<String> delete(String deletePath, int version) {
-    return delegate.delete(deletePath, version);
+  public OperationFuture<Stat> setACL(String path, Iterable<ACL> acl, int version) {
+    return delegate.setACL(path, acl, version);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/NodeData.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/NodeData.java b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/NodeData.java
index ac15957..a9bd247 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/NodeData.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/NodeData.java
@@ -22,7 +22,7 @@ import org.apache.zookeeper.data.Stat;
 import javax.annotation.Nullable;
 
 /**
- * Represents result of call to {@link ZKClientService#getData(String, org.apache.zookeeper.Watcher)}.
+ * Represents result of call to {@link ZKClient#getData(String, org.apache.zookeeper.Watcher)}.
  */
 public interface NodeData {
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/RetryStrategy.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/RetryStrategy.java b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/RetryStrategy.java
index 3301e8a..fe174b2 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/RetryStrategy.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/RetryStrategy.java
@@ -31,7 +31,9 @@ public interface RetryStrategy {
     GET_CHILDREN,
     GET_DATA,
     SET_DATA,
-    DELETE
+    DELETE,
+    SET_ACL,
+    GET_ACL
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClient.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClient.java b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClient.java
index d60182e..8f8d6b2 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClient.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClient.java
@@ -19,6 +19,7 @@ package org.apache.twill.zookeeper;
 
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 
 import javax.annotation.Nullable;
@@ -46,7 +47,7 @@ public interface ZKClient {
   void addConnectionWatcher(Watcher watcher);
 
   /**
-   * Same as calling
+   * Creates a path in zookeeper. Same as calling
    * {@link #create(String, byte[], org.apache.zookeeper.CreateMode, boolean) create(path, data, createMode, true)}.
    *
    * @see #create(String, byte[], org.apache.zookeeper.CreateMode, boolean)
@@ -54,17 +55,40 @@ public interface ZKClient {
   OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode);
 
   /**
+   * Creates a path in zookeeper. Same as calling
+   *
+   * {@link #create(String, byte[], org.apache.zookeeper.CreateMode, boolean, Iterable)
+   * create(path, data, createMode, createParent, ZooDefs.Ids.OPEN_ACL_UNSAFE)}
+   *
+   * @see #create(String, byte[], org.apache.zookeeper.CreateMode, boolean, Iterable)
+   */
+  OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode, boolean createParent);
+
+  /**
+   * Creates a path in zookeeper. Same as calling
+   *
+   * {@link #create(String, byte[], org.apache.zookeeper.CreateMode, boolean, Iterable)
+   * create(path, data, createMode, true, acl)}
+   *
+   * @see #create(String, byte[], org.apache.zookeeper.CreateMode, boolean, Iterable)
+   */
+  OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode, Iterable<ACL> acl);
+
+  /**
    * Creates a path in zookeeper, with given data and create mode.
    *
    * @param path Path to be created
    * @param data Data to be stored in the node, or {@code null} if no data to store.
    * @param createMode The {@link org.apache.zookeeper.CreateMode} for the node.
    * @param createParent If {@code true} and parent nodes are missing, it will create all parent nodes as normal
-   *                     persistent node before creating the request node.
+   *                     persistent node with the ACL {@link org.apache.zookeeper.ZooDefs.Ids#OPEN_ACL_UNSAFE}
+   *                     before creating the request node.
+   * @param acl Set of {@link ACL} to be set for the node being created.
    * @return A {@link OperationFuture} that will be completed when the
    *         creation is done. If there is error during creation, it will be reflected as error in the future.
    */
-  OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode, boolean createParent);
+  OperationFuture<String> create(String path, @Nullable byte[] data,
+                                 CreateMode createMode, boolean createParent, Iterable<ACL> acl);
 
   /**
    * Checks if the path exists. Same as calling
@@ -158,4 +182,32 @@ public interface ZKClient {
    *         given as the future result. If there is error, it will be reflected as error in the future.
    */
   OperationFuture<String> delete(String deletePath, int version);
+
+  /**
+   * Retrieves the Stat and ACL being set at the given path.
+   *
+   * @param path The path to get information from.
+   * @return A {@link OperationFuture} that will be completed when the getACL call is done, with the result given as
+   *         {@link ACLData}. If there is error, it will be reflected as error in the future.
+   */
+  OperationFuture<ACLData> getACL(String path);
+
+  /**
+   * Sets the ACL of the given path if the path exists. Same as calling
+   * {@link #setACL(String, Iterable, int) setACL(path, acl, -1)}
+   *
+   * @see #setACL(String, Iterable, int)
+   */
+  OperationFuture<Stat> setACL(String path, Iterable<ACL> acl);
+
+  /**
+   * Sets the ACL of the given path if the path exists and version matched.
+   *
+   * @param path The path to have ACL being set.
+   * @param acl ACL to set to.
+   * @param version Version of the node.
+   * @return A {@link OperationFuture} that will be completed when the setACL call is done, with the node {@link Stat}
+   *         available as the future result. If there is error, it will be reflected as error in the future.
+   */
+  OperationFuture<Stat> setACL(String path, Iterable<ACL> acl, int version);
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientService.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientService.java b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientService.java
index 8d159af..33c23aa 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientService.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientService.java
@@ -18,13 +18,12 @@
 package org.apache.twill.zookeeper;
 
 import com.google.common.base.Supplier;
-import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
 import com.google.common.util.concurrent.Service;
 import org.apache.twill.internal.zookeeper.DefaultZKClientService;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.ACL;
 
 /**
  * A {@link ZKClient} that extends from {@link Service} to provide lifecycle management functions.
@@ -50,7 +49,7 @@ public interface ZKClientService extends ZKClient, Service {
     private final String connectStr;
     private int timeout = 10000;
     private Watcher connectionWatcher;
-    private Multimap<String, ACL> acls = HashMultimap.create();
+    private Multimap<String, byte[]> auths = ArrayListMultimap.create();
 
     /**
      * Creates a {@link Builder} with the given ZooKeeper connection string.
@@ -82,11 +81,23 @@ public interface ZKClientService extends ZKClient, Service {
     }
 
     /**
+     * Adds an authorization information.
+     *
+     * @param schema The authorization schema.
+     * @param auth The authorization bytes.
+     * @return This builder.
+     */
+    public Builder addAuthInfo(String schema, byte[] auth) {
+      this.auths.put(schema, auth);
+      return this;
+    }
+
+    /**
      * Creates an instance of {@link ZKClientService} with the settings of this builder.
      * @return A new instance of {@link ZKClientService}.
      */
     public ZKClientService build() {
-      return new DefaultZKClientService(connectStr, timeout, connectionWatcher);
+      return new DefaultZKClientService(connectStr, timeout, connectionWatcher, auths);
     }
 
     private Builder(String connectStr) {

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientServices.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientServices.java b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientServices.java
index cc38c76..7cdfd36 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientServices.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/zookeeper/ZKClientServices.java
@@ -19,6 +19,7 @@ package org.apache.twill.zookeeper;
 
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 
 import javax.annotation.Nullable;
@@ -77,20 +78,9 @@ public final class ZKClientServices {
         client.addConnectionWatcher(watcher);
       }
 
-      @Override
-      public OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode) {
-        return client.create(path, data, createMode);
-      }
-
-      @Override
-      public OperationFuture<String> create(String path, @Nullable byte[] data, CreateMode createMode,
-                                            boolean createParent) {
-        return client.create(path, data, createMode, createParent);
-      }
-
-      @Override
-      public OperationFuture<Stat> exists(String path) {
-        return client.exists(path);
+      public OperationFuture<String> create(String path, @Nullable byte[] data,
+                                            CreateMode createMode, boolean createParent, Iterable<ACL> acl) {
+        return client.create(path, data, createMode, createParent, acl);
       }
 
       @Override
@@ -99,44 +89,32 @@ public final class ZKClientServices {
       }
 
       @Override
-      public OperationFuture<NodeChildren> getChildren(String path) {
-        return client.getChildren(path);
-      }
-
-      @Override
       public OperationFuture<NodeChildren> getChildren(String path, @Nullable Watcher watcher) {
         return client.getChildren(path, watcher);
       }
 
       @Override
-      public OperationFuture<NodeData> getData(String path) {
-        return client.getData(path);
-      }
-
-      @Override
       public OperationFuture<NodeData> getData(String path, @Nullable Watcher watcher) {
         return client.getData(path, watcher);
       }
 
       @Override
-      public OperationFuture<Stat> setData(String path, byte[] data) {
-        return client.setData(path, data);
-      }
-
-      @Override
       public OperationFuture<Stat> setData(String dataPath, byte[] data, int version) {
         return client.setData(dataPath, data, version);
       }
 
       @Override
-      public OperationFuture<String> delete(String path) {
-        return client.delete(path);
-      }
-
-      @Override
       public OperationFuture<String> delete(String deletePath, int version) {
         return client.delete(deletePath, version);
       }
+
+      public OperationFuture<ACLData> getACL(String path) {
+        return client.getACL(path);
+      }
+
+      public OperationFuture<Stat> setACL(String path, Iterable<ACL> acl, int version) {
+        return client.setACL(path, acl, version);
+      }
     };
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9393df80/twill-zookeeper/src/test/java/org/apache/twill/zookeeper/ZKClientTest.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/test/java/org/apache/twill/zookeeper/ZKClientTest.java b/twill-zookeeper/src/test/java/org/apache/twill/zookeeper/ZKClientTest.java
index 2228d46..40b0a39 100644
--- a/twill-zookeeper/src/test/java/org/apache/twill/zookeeper/ZKClientTest.java
+++ b/twill-zookeeper/src/test/java/org/apache/twill/zookeeper/ZKClientTest.java
@@ -20,19 +20,25 @@ package org.apache.twill.zookeeper;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
-import com.google.common.io.Files;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import org.apache.twill.internal.zookeeper.InMemoryZKServer;
 import org.apache.twill.internal.zookeeper.KillZKSession;
 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;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
 import org.junit.Assert;
+import org.junit.ClassRule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 
 import java.io.File;
 import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.BlockingQueue;
@@ -48,6 +54,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
  */
 public class ZKClientTest {
 
+  @ClassRule
+  public static TemporaryFolder tmpFolder = new TemporaryFolder();
+
   @Test
   public void testChroot() throws Exception {
     InMemoryZKServer zkServer = InMemoryZKServer.builder().setTickTime(1000).build();
@@ -201,8 +210,8 @@ public class ZKClientTest {
   }
 
   @Test
-  public void testRetry() throws ExecutionException, InterruptedException, TimeoutException {
-    File dataDir = Files.createTempDir();
+  public void testRetry() throws ExecutionException, InterruptedException, TimeoutException, IOException {
+    File dataDir = tmpFolder.newFolder();
     InMemoryZKServer zkServer = InMemoryZKServer.builder().setDataDir(dataDir).setTickTime(1000).build();
     zkServer.startAndWait();
     int port = zkServer.getLocalAddress().getPort();
@@ -251,4 +260,64 @@ public class ZKClientTest {
       zkServer.stopAndWait();
     }
   }
+
+  @Test
+  public void testACL() throws IOException, ExecutionException, InterruptedException, NoSuchAlgorithmException {
+    InMemoryZKServer zkServer = InMemoryZKServer.builder().setDataDir(tmpFolder.newFolder()).setTickTime(1000).build();
+    zkServer.startAndWait();
+
+    try {
+      String userPass = "user:pass";
+      String digest = DigestAuthenticationProvider.generateDigest(userPass);
+
+      // Creates two zkclients
+      ZKClientService zkClient = ZKClientService.Builder
+                                                .of(zkServer.getConnectionStr())
+                                                .addAuthInfo("digest", userPass.getBytes())
+                                                .build();
+      zkClient.startAndWait();
+
+      ZKClientService noAuthClient = ZKClientService.Builder.of(zkServer.getConnectionStr()).build();
+      noAuthClient.startAndWait();
+
+
+      // Create a node that is readable by all client, but admin for the creator
+      String path = "/testacl";
+      zkClient.create(path, "test".getBytes(), CreateMode.PERSISTENT,
+                      ImmutableList.of(
+                        new ACL(ZooDefs.Perms.READ, ZooDefs.Ids.ANYONE_ID_UNSAFE),
+                        new ACL(ZooDefs.Perms.ALL, ZooDefs.Ids.AUTH_IDS)
+                      )).get();
+
+      // Verify the ACL
+      ACLData aclData = zkClient.getACL(path).get();
+      Assert.assertEquals(2, aclData.getACL().size());
+      ACL acl = aclData.getACL().get(1);
+      Assert.assertEquals(ZooDefs.Perms.ALL, acl.getPerms());
+      Assert.assertEquals("digest", acl.getId().getScheme());
+      Assert.assertEquals(digest, acl.getId().getId());
+
+      Assert.assertEquals("test", new String(noAuthClient.getData(path).get().getData()));
+
+      // When tries to write using the no-auth zk client, it should fail.
+      try {
+        noAuthClient.setData(path, "test2".getBytes()).get();
+        Assert.fail();
+      } catch (ExecutionException e) {
+        Assert.assertTrue(e.getCause() instanceof KeeperException.NoAuthException);
+      }
+
+      // Change ACL to make it open for all
+      zkClient.setACL(path, ImmutableList.of(new ACL(ZooDefs.Perms.WRITE, ZooDefs.Ids.ANYONE_ID_UNSAFE))).get();
+
+      // Write again with the non-auth client, now should succeed.
+      noAuthClient.setData(path, "test2".getBytes()).get();
+
+      noAuthClient.stopAndWait();
+      zkClient.stopAndWait();
+
+    } finally {
+      zkServer.stopAndWait();
+    }
+  }
 }


[20/50] [abbrv] git commit: Bump version to 0.2.0-incubating

Posted by ch...@apache.org.
Bump version to 0.2.0-incubating

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

Branch: refs/heads/site
Commit: 4ecdfe1ff89e2c0e4c372f08b659d3d571da22cd
Parents: bb24c02
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Feb 10 10:24:22 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Feb 10 10:24:22 2014 -0800

----------------------------------------------------------------------
 pom.xml                      | 2 +-
 twill-api/pom.xml            | 2 +-
 twill-common/pom.xml         | 2 +-
 twill-core/pom.xml           | 2 +-
 twill-discovery-api/pom.xml  | 2 +-
 twill-discovery-core/pom.xml | 2 +-
 twill-yarn/pom.xml           | 2 +-
 twill-zookeeper/pom.xml      | 2 +-
 8 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/4ecdfe1f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index df87929..1d0a816 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
 
     <groupId>org.apache.twill</groupId>
     <artifactId>twill-parent</artifactId>
-    <version>0.1.0-incubating-SNAPSHOT</version>
+    <version>0.2.0-incubating-SNAPSHOT</version>
     <packaging>pom</packaging>
     <name>Apache Twill</name>
     <url>http://twill.incubator.apache.org</url>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/4ecdfe1f/twill-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-api/pom.xml b/twill-api/pom.xml
index a018adf..338a716 100644
--- a/twill-api/pom.xml
+++ b/twill-api/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.twill</groupId>
         <artifactId>twill-parent</artifactId>
-        <version>0.1.0-incubating-SNAPSHOT</version>
+        <version>0.2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <artifactId>twill-api</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/4ecdfe1f/twill-common/pom.xml
----------------------------------------------------------------------
diff --git a/twill-common/pom.xml b/twill-common/pom.xml
index c39c87a..7506d84 100644
--- a/twill-common/pom.xml
+++ b/twill-common/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-incubating-SNAPSHOT</version>
+        <version>0.2.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/4ecdfe1f/twill-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-core/pom.xml b/twill-core/pom.xml
index 6bf4e0f..859c653 100644
--- a/twill-core/pom.xml
+++ b/twill-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-incubating-SNAPSHOT</version>
+        <version>0.2.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/4ecdfe1f/twill-discovery-api/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-api/pom.xml b/twill-discovery-api/pom.xml
index 78b5d26..2a12f39 100644
--- a/twill-discovery-api/pom.xml
+++ b/twill-discovery-api/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-incubating-SNAPSHOT</version>
+        <version>0.2.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/4ecdfe1f/twill-discovery-core/pom.xml
----------------------------------------------------------------------
diff --git a/twill-discovery-core/pom.xml b/twill-discovery-core/pom.xml
index 4531fbb..5570986 100644
--- a/twill-discovery-core/pom.xml
+++ b/twill-discovery-core/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-incubating-SNAPSHOT</version>
+        <version>0.2.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/4ecdfe1f/twill-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-yarn/pom.xml b/twill-yarn/pom.xml
index b1493e8..0c94376 100644
--- a/twill-yarn/pom.xml
+++ b/twill-yarn/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-incubating-SNAPSHOT</version>
+        <version>0.2.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/4ecdfe1f/twill-zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/twill-zookeeper/pom.xml b/twill-zookeeper/pom.xml
index a9d7324..70de6bf 100644
--- a/twill-zookeeper/pom.xml
+++ b/twill-zookeeper/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>twill-parent</artifactId>
         <groupId>org.apache.twill</groupId>
-        <version>0.1.0-incubating-SNAPSHOT</version>
+        <version>0.2.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 


[30/50] [abbrv] git commit: fixing checkstyle violations #TWILL-40

Posted by ch...@apache.org.
fixing checkstyle violations #TWILL-40

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: fd10dc9557eb9e7a1a14b4bdb84287093f27c7cf
Parents: ac67f1f
Author: Fabian Murariu <mu...@gmail.com>
Authored: Fri Feb 14 11:57:01 2014 +0200
Committer: Terence Yim <te...@continuuity.com>
Committed: Fri Feb 21 12:33:26 2014 -0800

----------------------------------------------------------------------
 .../apache/twill/api/ResourceSpecification.java |  4 +-
 .../internal/DefaultResourceSpecification.java  |  4 +-
 .../json/ResourceSpecificationCodec.java        |  8 +++-
 .../json/ResourceSpecificationCodecTest.java    | 40 +++++++++++++-------
 .../twill/internal/yarn/YarnAMClient.java       | 11 ++++--
 .../apache/twill/yarn/EchoServerTestRun.java    |  2 -
 6 files changed, 45 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fd10dc95/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
index 75427a7..5416346 100644
--- a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
@@ -79,14 +79,14 @@ public interface ResourceSpecification {
   int getInstances();
 
   /**
-   * Returns the execution hosts, expects Fully Qualified Domain Names host + domain
+   * Returns the execution hosts, expects Fully Qualified Domain Names host + domain.
    * This is a suggestion for the scheduler depending on cluster load it may ignore it
    * @return An array containing the hosts where the containers should run
    */
   String[] getHosts();
 
   /**
-   * Returns the execution racks
+   * Returns the execution racks.
    * This is a suggestion for the scheduler depending on cluster load it may ignore it
    * @return An array containing the racks where the containers should run
    */

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fd10dc95/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java b/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
index 4451ac1..7274b5c 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
@@ -35,7 +35,9 @@ public final class DefaultResourceSpecification implements ResourceSpecification
     this(virtualCores, memorySize, instances, uplink, downlink, new String[0], new String[0]);
   }
 
-  public DefaultResourceSpecification(int virtualCores, int memorySize, int instances, int uplink, int downlink, String[] hosts, String[] racks) {
+  public DefaultResourceSpecification(int virtualCores, int memorySize, int instances,
+                                      int uplink, int downlink,
+                                      String[] hosts, String[] racks) {
     this.virtualCores = virtualCores;
     this.memorySize = memorySize;
     this.instances = instances;

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fd10dc95/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
index e88045c..5233220 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
@@ -17,7 +17,13 @@
  */
 package org.apache.twill.internal.json;
 
-import com.google.gson.*;
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParseException;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
 import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.internal.DefaultResourceSpecification;
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fd10dc95/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java b/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
index 7bd4f39..b5ffec6 100644
--- a/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
+++ b/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
@@ -5,12 +5,13 @@ import com.google.gson.GsonBuilder;
 import com.google.gson.JsonElement;
 import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.internal.DefaultResourceSpecification;
+import org.junit.Assert;
 import org.junit.Test;
+import org.unitils.reflectionassert.ReflectionAssert;
 
-import static org.apache.twill.api.ResourceSpecification.SizeUnit.GIGA;
-import static org.junit.Assert.assertEquals;
-import static org.unitils.reflectionassert.ReflectionAssert.assertLenientEquals;
-
+/**
+ * Maybe this checkstyle rule needs to be removed
+ */
 public class ResourceSpecificationCodecTest {
   private final Gson gson = new GsonBuilder().serializeNulls()
           .registerTypeAdapter(ResourceSpecification.class, new ResourceSpecificationCodec())
@@ -19,33 +20,44 @@ public class ResourceSpecificationCodecTest {
 
   @Test
   public void testCodec() throws Exception {
-    String expectedString = "{\"cores\":2,\"memorySize\":1024,\"instances\":2,\"uplink\":100,\"downlink\":100,\"hosts\":[\"one1\",\"two2\"],\"racks\":[\"three3\"]}";
+    String expectedString =
+            "{" +
+                    "\"cores\":2," +
+                    "\"memorySize\":1024," +
+                    "\"instances\":2," +
+                    "\"uplink\":100," +
+                    "\"downlink\":100," +
+                    "\"hosts\":[\"one1\",\"two2\"]," +
+                    "\"racks\":[\"three3\"]" +
+            "}";
     final ResourceSpecification expected =
-            new DefaultResourceSpecification(2, 1024, 2, 100, 100, new String[]{"one1", "two2"}, new String[]{"three3"});
+            new DefaultResourceSpecification(2, 1024, 2, 100, 100,
+                    new String[]{"one1", "two2"}, new String[]{"three3"});
     final String actualString = gson.toJson(expected);
-    assertEquals(expectedString, actualString);
+    Assert.assertEquals(expectedString, actualString);
 
     final JsonElement expectedJson = gson.toJsonTree(expected);
     final ResourceSpecification actual = gson.fromJson(expectedJson, DefaultResourceSpecification.class);
     final JsonElement actualJson = gson.toJsonTree(actual);
 
-    assertEquals(expectedJson, actualJson);
-    assertLenientEquals(expected, actual);
+    Assert.assertEquals(expectedJson, actualJson);
+    ReflectionAssert.assertLenientEquals(expected, actual);
   }
 
   @Test
   public void testBuilder() throws Exception {
     final ResourceSpecification actual = ResourceSpecification.Builder.with()
             .setVirtualCores(5)
-            .setMemory(4, GIGA)
+            .setMemory(4, ResourceSpecification.SizeUnit.GIGA)
             .setInstances(3)
             .setHosts("a1", "b2", "c3")
             .setRacks("r2")
-            .setUplink(10, GIGA)
-            .setDownlink(5, GIGA).build();
+            .setUplink(10, ResourceSpecification.SizeUnit.GIGA)
+            .setDownlink(5, ResourceSpecification.SizeUnit.GIGA).build();
     final DefaultResourceSpecification expectd =
-            new DefaultResourceSpecification(5, 4096, 3, 10240, 5120, new String[]{"a1", "b2", "c3"}, new String[]{"r2"});
-    assertLenientEquals(expectd, actual);
+            new DefaultResourceSpecification(5, 4096, 3, 10240, 5120,
+                    new String[]{"a1", "b2", "c3"}, new String[]{"r2"});
+    ReflectionAssert.assertLenientEquals(expectd, actual);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fd10dc95/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
index c8da649..9351201 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
@@ -19,6 +19,7 @@ package org.apache.twill.internal.yarn;
 
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.Service;
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -27,9 +28,10 @@ import org.apache.twill.internal.ProcessLauncher;
 
 import java.net.InetSocketAddress;
 import java.net.URL;
-import java.util.*;
-
-import static org.apache.commons.lang.ArrayUtils.isEmpty;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
 
 /**
  * This interface provides abstraction for AM to interacts with YARN to abstract out YARN version specific
@@ -72,8 +74,9 @@ public interface YarnAMClient extends Service {
     public abstract String apply();
 
     private <T> ContainerRequestBuilder add(Collection<T> collection, T... more) {
-      if (!isEmpty(more))
+      if (!ArrayUtils.isEmpty(more)){
         Collections.addAll(collection, more);
+      }
       return this;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/fd10dc95/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
index 1c3034c..23fc82b 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/EchoServerTestRun.java
@@ -61,8 +61,6 @@ public final class EchoServerTestRun extends BaseYarnTest {
                                                          .setVirtualCores(1)
                                                          .setMemory(1, ResourceSpecification.SizeUnit.GIGA)
                                                          .setInstances(2)
-                                                         .setHosts("someHost1.domain.no","someHost2.domain.no") /*demo only ignored in this test*/
-                                                         .setRacks("someRack1") /* demo only ignored in this test*/
                                                          .build())
                                         .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
                                         .withApplicationArguments("echo")


[16/50] [abbrv] git commit: [TWILL-36] Added assembly target for building source tarball for release.

Posted by ch...@apache.org.
[TWILL-36] Added assembly target for building source tarball for release.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: ed41c444b4b3fabb52e4556a044ab651cbe7c33a
Parents: a28f3c0
Author: Terence Yim <te...@continuuity.com>
Authored: Tue Jan 28 11:29:34 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Tue Jan 28 12:29:36 2014 -0800

----------------------------------------------------------------------
 pom.xml | 18 ++++++++++++++++++
 1 file changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/ed41c444/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 01c0648..f960b75 100644
--- a/pom.xml
+++ b/pom.xml
@@ -334,6 +334,24 @@
 
     <profiles>
         <profile>
+            <id>apache-release</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-assembly-plugin</artifactId>
+                        <version>2.4</version>
+                        <configuration>
+                            <finalName>twill-${project.version}</finalName>
+                            <formats>
+                                <format>tar.gz</format>
+                            </formats>
+                        </configuration>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+        <profile>
             <id>hadoop-2.0</id>
             <properties>
                 <hadoop.version>2.0.2-alpha</hadoop.version>


[21/50] [abbrv] git commit: TWILL-41: Log host name where container was launched

Posted by ch...@apache.org.
TWILL-41: Log host name where container was launched

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: d9e8ead2e9b40bbf4973152a07724ac55deeb5b8
Parents: 4ecdfe1
Author: Alex Baranau <al...@continuuity.com>
Authored: Mon Feb 10 10:44:27 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Feb 10 10:49:23 2014 -0800

----------------------------------------------------------------------
 .../apache/twill/internal/appmaster/RunnableProcessLauncher.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/d9e8ead2/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableProcessLauncher.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableProcessLauncher.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableProcessLauncher.java
index 29b3f9c..2126541 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableProcessLauncher.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunnableProcessLauncher.java
@@ -68,7 +68,8 @@ public final class RunnableProcessLauncher extends AbstractYarnProcessLauncher<Y
 
     launchContext.setEnvironment(env);
 
-    LOG.info("Launching in container {}, {}", containerInfo.getId(), launchContext.getCommands());
+    LOG.info("Launching in container {} at {}, {}",
+             containerInfo.getId(), containerInfo.getHost(), launchContext.getCommands());
     final Cancellable cancellable = nmClient.start(containerInfo, launchContext);
     launched = true;
 


[31/50] [abbrv] git commit: made changes as per review #18209 for #TWILL-40

Posted by ch...@apache.org.
made changes as per review #18209 for #TWILL-40

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: e3ccab184dcea843fa12b877b238a2ca0256f906
Parents: fd10dc9
Author: Fabian Murariu <mu...@gmail.com>
Authored: Wed Feb 19 17:03:49 2014 +0200
Committer: Terence Yim <te...@continuuity.com>
Committed: Fri Feb 21 12:33:26 2014 -0800

----------------------------------------------------------------------
 .../apache/twill/api/ResourceSpecification.java | 71 +++++++++++++-------
 .../internal/DefaultResourceSpecification.java  | 20 +++---
 .../json/ResourceSpecificationCodec.java        |  7 +-
 .../json/ResourceSpecificationCodecTest.java    | 30 +++++++--
 .../twill/internal/yarn/YarnAMClient.java       | 10 +--
 5 files changed, 90 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e3ccab18/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
index 5416346..b37e491 100644
--- a/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/api/ResourceSpecification.java
@@ -19,6 +19,11 @@ package org.apache.twill.api;
 
 import org.apache.twill.internal.DefaultResourceSpecification;
 
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
 /**
  * This interface provides specifications for resource requirements including set and get methods
  * for number of cores, amount of memory, and number of instances.
@@ -83,14 +88,14 @@ public interface ResourceSpecification {
    * This is a suggestion for the scheduler depending on cluster load it may ignore it
    * @return An array containing the hosts where the containers should run
    */
-  String[] getHosts();
+  List<String> getHosts();
 
   /**
    * Returns the execution racks.
    * This is a suggestion for the scheduler depending on cluster load it may ignore it
    * @return An array containing the racks where the containers should run
    */
-  String[] getRacks();
+  List<String> getRacks();
 
   /**
    * Builder for creating {@link ResourceSpecification}.
@@ -102,31 +107,13 @@ public interface ResourceSpecification {
     private int uplink = -1;
     private int downlink = -1;
     private int instances = 1;
-    private String[] hosts = new String[0];
-    private String[] racks = new String[0];
+    private List<String> hosts = new LinkedList<String>();
+    private List<String> racks = new LinkedList<String>();
 
     public static CoreSetter with() {
       return new Builder().new CoreSetter();
     }
 
-    public final class HostsSetter extends Build {
-      public RackSetter setHosts(String... hosts) {
-        if (hosts != null) {
-          Builder.this.hosts = hosts.clone();
-        }
-        return new RackSetter();
-      }
-    }
-
-    public final class RackSetter extends Build {
-      public AfterRacks setRacks(String... racks) {
-        if (racks != null) {
-          Builder.this.racks = racks.clone();
-        }
-        return new AfterRacks();
-      }
-    }
-
     public final class CoreSetter {
       @Deprecated
       public MemorySetter setCores(int cores) {
@@ -148,13 +135,13 @@ public interface ResourceSpecification {
     }
 
     public final class AfterMemory extends Build {
-      public HostsSetter setInstances(int instances) {
+      public AfterInstances setInstances(int instances) {
         Builder.this.instances = instances;
-        return new HostsSetter();
+        return new AfterInstances();
       }
     }
 
-    public final class AfterRacks extends Build {
+    public final class AfterInstances extends Build {
       public AfterUplink setUplink(int uplink, SizeUnit unit) {
         Builder.this.uplink = uplink * unit.multiplier;
         return new AfterUplink();
@@ -162,10 +149,42 @@ public interface ResourceSpecification {
     }
 
     public final class AfterUplink extends Build {
-      public Done setDownlink(int downlink, SizeUnit unit) {
+      public AfterDownlink setDownlink(int downlink, SizeUnit unit) {
         Builder.this.downlink = downlink * unit.multiplier;
+        return new AfterDownlink();
+      }
+    }
+
+    public final class AfterHosts extends Build {
+      public Done setRacks(String... racks) {
+        if (racks != null) {
+          Builder.this.racks = Arrays.asList(racks);
+        }
         return new Done();
       }
+
+      public Done setRacks(Collection<String> racks){
+        if (racks != null){
+          Builder.this.racks.addAll(racks);
+        }
+        return new Done();
+      }
+    }
+
+    public final class AfterDownlink extends Build {
+      public AfterHosts setHosts(String... hosts) {
+        if (hosts != null) {
+          Builder.this.hosts = Arrays.asList(hosts);
+        }
+        return new AfterHosts();
+      }
+
+      public AfterHosts setHosts(Collection<String> hosts){
+        if (hosts != null){
+          Builder.this.hosts.addAll(hosts);
+        }
+        return new AfterHosts();
+      }
     }
 
     public final class Done extends Build {

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e3ccab18/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java b/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
index 7274b5c..3b1cc26 100644
--- a/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
+++ b/twill-api/src/main/java/org/apache/twill/internal/DefaultResourceSpecification.java
@@ -19,6 +19,9 @@ package org.apache.twill.internal;
 
 import org.apache.twill.api.ResourceSpecification;
 
+import java.util.Collections;
+import java.util.List;
+
 /**
  * Straightforward implementation of {@link org.apache.twill.api.ResourceSpecification}.
  */
@@ -28,16 +31,17 @@ public final class DefaultResourceSpecification implements ResourceSpecification
   private final int instances;
   private final int uplink;
   private final int downlink;
-  private final String[] hosts;
-  private final String[] racks;
+  private final List<String> hosts;
+  private final List<String> racks;
 
   public DefaultResourceSpecification(int virtualCores, int memorySize, int instances, int uplink, int downlink) {
-    this(virtualCores, memorySize, instances, uplink, downlink, new String[0], new String[0]);
+    this(virtualCores, memorySize, instances, uplink, downlink,
+            Collections.<String>emptyList(), Collections.<String>emptyList());
   }
 
   public DefaultResourceSpecification(int virtualCores, int memorySize, int instances,
                                       int uplink, int downlink,
-                                      String[] hosts, String[] racks) {
+                                      List<String> hosts, List<String> racks) {
     this.virtualCores = virtualCores;
     this.memorySize = memorySize;
     this.instances = instances;
@@ -69,13 +73,13 @@ public final class DefaultResourceSpecification implements ResourceSpecification
   }
 
   @Override
-  public String[] getHosts() {
-    return hosts.clone();
+  public List<String> getHosts() {
+    return Collections.unmodifiableList(this.hosts);
   }
 
   @Override
-  public String[] getRacks() {
-    return racks.clone();
+  public List<String> getRacks() {
+    return Collections.unmodifiableList(this.racks);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e3ccab18/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
index 5233220..5f7d7ae 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/json/ResourceSpecificationCodec.java
@@ -28,6 +28,7 @@ import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.internal.DefaultResourceSpecification;
 
 import java.lang.reflect.Type;
+import java.util.List;
 
 /**
  *
@@ -52,8 +53,8 @@ final class ResourceSpecificationCodec implements JsonSerializer<ResourceSpecifi
   public ResourceSpecification deserialize(JsonElement json, Type typeOfT,
                                            JsonDeserializationContext context) throws JsonParseException {
     JsonObject jsonObj = json.getAsJsonObject();
-    final String[] hosts = context.deserialize(jsonObj.getAsJsonArray("hosts"), String[].class);
-    final String[] racks = context.deserialize(jsonObj.getAsJsonArray("racks"), String[].class);
+    final List<String> hosts = context.deserialize(jsonObj.getAsJsonArray("hosts"), List.class);
+    final List<String> racks = context.deserialize(jsonObj.getAsJsonArray("racks"), List.class);
     return new DefaultResourceSpecification(jsonObj.get("cores").getAsInt(),
                                             jsonObj.get("memorySize").getAsInt(),
                                             jsonObj.get("instances").getAsInt(),
@@ -61,8 +62,6 @@ final class ResourceSpecificationCodec implements JsonSerializer<ResourceSpecifi
                                             jsonObj.get("downlink").getAsInt(),
                                             hosts,
                                             racks);
-
-
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e3ccab18/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java b/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
index b5ffec6..044a465 100644
--- a/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
+++ b/twill-core/src/test/java/org/apache/twill/internal/json/ResourceSpecificationCodecTest.java
@@ -9,6 +9,8 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.unitils.reflectionassert.ReflectionAssert;
 
+import java.util.Arrays;
+
 /**
  * Maybe this checkstyle rule needs to be removed
  */
@@ -32,7 +34,7 @@ public class ResourceSpecificationCodecTest {
             "}";
     final ResourceSpecification expected =
             new DefaultResourceSpecification(2, 1024, 2, 100, 100,
-                    new String[]{"one1", "two2"}, new String[]{"three3"});
+                    Arrays.asList("one1", "two2"), Arrays.asList("three3"));
     final String actualString = gson.toJson(expected);
     Assert.assertEquals(expectedString, actualString);
 
@@ -50,14 +52,32 @@ public class ResourceSpecificationCodecTest {
             .setVirtualCores(5)
             .setMemory(4, ResourceSpecification.SizeUnit.GIGA)
             .setInstances(3)
+            .setUplink(10, ResourceSpecification.SizeUnit.GIGA)
+            .setDownlink(5, ResourceSpecification.SizeUnit.GIGA)
             .setHosts("a1", "b2", "c3")
             .setRacks("r2")
+            .build();
+    final DefaultResourceSpecification expected =
+            new DefaultResourceSpecification(5, 4096, 3, 10240, 5120,
+                    Arrays.asList("a1", "b2", "c3"), Arrays.asList("r2"));
+    ReflectionAssert.assertLenientEquals(expected, actual);
+  }
+
+  @Test
+  public void testBuilderWithLists() throws Exception {
+    final ResourceSpecification actual = ResourceSpecification.Builder.with()
+            .setVirtualCores(5)
+            .setMemory(4, ResourceSpecification.SizeUnit.GIGA)
+            .setInstances(3)
             .setUplink(10, ResourceSpecification.SizeUnit.GIGA)
-            .setDownlink(5, ResourceSpecification.SizeUnit.GIGA).build();
-    final DefaultResourceSpecification expectd =
+            .setDownlink(5, ResourceSpecification.SizeUnit.GIGA)
+            .setHosts(Arrays.asList("a1", "b2", "c3"))
+            .setRacks(Arrays.asList("r2"))
+            .build();
+    final DefaultResourceSpecification expected =
             new DefaultResourceSpecification(5, 4096, 3, 10240, 5120,
-                    new String[]{"a1", "b2", "c3"}, new String[]{"r2"});
-    ReflectionAssert.assertLenientEquals(expectd, actual);
+                    Arrays.asList("a1", "b2", "c3"), Arrays.asList("r2"));
+    ReflectionAssert.assertLenientEquals(expected, actual);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/e3ccab18/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
index 9351201..370ca3c 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/yarn/YarnAMClient.java
@@ -55,11 +55,11 @@ public interface YarnAMClient extends Service {
       this.count = count;
     }
 
-    public ContainerRequestBuilder addHosts(String...newHosts) {
+    public ContainerRequestBuilder addHosts(Collection<String> newHosts) {
       return add(hosts, newHosts);
     }
 
-    public ContainerRequestBuilder addRacks(String...newRacks) {
+    public ContainerRequestBuilder addRacks(Collection<String> newRacks) {
       return add(racks, newRacks);
     }
 
@@ -73,9 +73,9 @@ public interface YarnAMClient extends Service {
      */
     public abstract String apply();
 
-    private <T> ContainerRequestBuilder add(Collection<T> collection, T... more) {
-      if (!ArrayUtils.isEmpty(more)){
-        Collections.addAll(collection, more);
+    private <T> ContainerRequestBuilder add(Collection<T> collection, Collection<T> more) {
+      if (more != null){
+        collection.addAll(more);
       }
       return this;
     }


[25/50] [abbrv] git commit: pulled dependencies versions up into maven properties for better overview in parrent #TWILL-39

Posted by ch...@apache.org.
pulled dependencies versions up into maven properties for better overview in parrent #TWILL-39

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 9f66b86278bdf27d3d4c5c15ceb7da2f22385989
Parents: 652b91c
Author: Fabian Murariu <mu...@gmail.com>
Authored: Tue Feb 11 11:29:05 2014 +0200
Committer: Terence Yim <te...@continuuity.com>
Committed: Tue Feb 11 12:55:55 2014 -0800

----------------------------------------------------------------------
 pom.xml | 41 +++++++++++++++++++++++++++--------------
 1 file changed, 27 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/9f66b862/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4572d08..2141000 100644
--- a/pom.xml
+++ b/pom.xml
@@ -169,6 +169,19 @@
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <surefire.redirectTestOutputToFile>true</surefire.redirectTestOutputToFile>
+        <slf4j.version>1.7.5</slf4j.version>
+        <logback.version>1.0.9</logback.version>
+        <guava.version>13.0.1</guava.version>
+        <gson.version>2.2.4</gson.version>
+        <findbugs.jsr305.version>2.0.1</findbugs.jsr305.version>
+        <netty.version>3.6.6.Final</netty.version>
+        <snappy-java.version>1.0.5</snappy-java.version>
+        <jcl-over-slf4j.version>1.7.2</jcl-over-slf4j.version>
+        <asm.version>4.0</asm.version>
+        <kafka.version>0.8.0</kafka.version>
+        <zookeeper.version>3.4.5</zookeeper.version>
+        <junit.version>4.11</junit.version>
+        <commons-compress.version>1.5</commons-compress.version>
         <hadoop.version>[2.0.2-alpha,2.2.0]</hadoop.version>
         <hadoop20.output.dir>target/hadoop20-classes</hadoop20.output.dir>
     </properties>
@@ -525,22 +538,22 @@
             <dependency>
                 <groupId>com.google.guava</groupId>
                 <artifactId>guava</artifactId>
-                <version>13.0.1</version>
+                <version>${guava.version}</version>
             </dependency>
             <dependency>
                 <groupId>com.google.code.findbugs</groupId>
                 <artifactId>jsr305</artifactId>
-                <version>2.0.1</version>
+                <version>${findbugs.jsr305.version}</version>
             </dependency>
             <dependency>
                 <groupId>com.google.code.gson</groupId>
                 <artifactId>gson</artifactId>
-                <version>2.2.4</version>
+                <version>${gson.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.zookeeper</groupId>
                 <artifactId>zookeeper</artifactId>
-                <version>3.4.5</version>
+                <version>${zookeeper.version}</version>
                 <exclusions>
                     <exclusion>
                         <artifactId>slf4j-api</artifactId>
@@ -567,42 +580,42 @@
             <dependency>
                 <groupId>io.netty</groupId>
                 <artifactId>netty</artifactId>
-                <version>3.6.6.Final</version>
+                <version>${netty.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.xerial.snappy</groupId>
                 <artifactId>snappy-java</artifactId>
-                <version>1.0.5</version>
+                <version>${snappy-java.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.slf4j</groupId>
                 <artifactId>slf4j-api</artifactId>
-                <version>1.7.5</version>
+                <version>${slf4j.version}</version>
             </dependency>
             <dependency>
                 <groupId>ch.qos.logback</groupId>
                 <artifactId>logback-core</artifactId>
-                <version>1.0.9</version>
+                <version>${logback.version}</version>
             </dependency>
             <dependency>
                 <groupId>ch.qos.logback</groupId>
                 <artifactId>logback-classic</artifactId>
-                <version>1.0.9</version>
+                <version>${logback.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.slf4j</groupId>
                 <artifactId>jcl-over-slf4j</artifactId>
-                <version>1.7.2</version>
+                <version>${jcl-over-slf4j.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.ow2.asm</groupId>
                 <artifactId>asm-all</artifactId>
-                <version>4.0</version>
+                <version>${asm.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.kafka</groupId>
                 <artifactId>kafka_2.10</artifactId>
-                <version>0.8.0</version>
+                <version>${kafka.version}</version>
                 <exclusions>
                     <exclusion>
                         <groupId>org.slf4j</groupId>
@@ -780,13 +793,13 @@
             <dependency>
                 <groupId>junit</groupId>
                 <artifactId>junit</artifactId>
-                <version>4.11</version>
+                <version>${junit.version}</version>
                 <scope>test</scope>
             </dependency>
             <dependency>
                 <groupId>org.apache.commons</groupId>
                 <artifactId>commons-compress</artifactId>
-                <version>1.5</version>
+                <version>${commons-compress.version}</version>
                 <scope>test</scope>
             </dependency>
         </dependencies>


[45/50] [abbrv] git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-twill

Posted by ch...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-twill


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

Branch: refs/heads/site
Commit: 15f4a7e5b7fb4242ad57e002212d6ca66240779d
Parents: 1187ebf 6553ed0
Author: anew <an...@continuuity.com>
Authored: Mon Apr 14 12:17:56 2014 -0700
Committer: anew <an...@continuuity.com>
Committed: Mon Apr 14 12:17:56 2014 -0700

----------------------------------------------------------------------

----------------------------------------------------------------------



[29/50] [abbrv] git commit: (TWILL-48) Make AM not to re-launch container that failed at initialize stage.

Posted by ch...@apache.org.
(TWILL-48) Make AM not to re-launch container that failed at initialize stage.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: a77b67cf3e465b966487b3002cb1450590cbc9d3
Parents: 9171d22
Author: Terence Yim <te...@continuuity.com>
Authored: Fri Feb 14 13:44:48 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Feb 17 13:41:20 2014 -0800

----------------------------------------------------------------------
 .../org/apache/twill/api/TwillRunnable.java     |  4 +-
 .../twill/internal/ContainerExitCodes.java      | 51 +++++++++++++++
 .../org/apache/twill/internal/ServiceMain.java  | 27 +++++---
 .../internal/appmaster/RunningContainers.java   | 16 ++++-
 .../twill/yarn/InitializeFailTestRun.java       | 68 ++++++++++++++++++++
 5 files changed, 153 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/a77b67cf/twill-api/src/main/java/org/apache/twill/api/TwillRunnable.java
----------------------------------------------------------------------
diff --git a/twill-api/src/main/java/org/apache/twill/api/TwillRunnable.java b/twill-api/src/main/java/org/apache/twill/api/TwillRunnable.java
index 4350bfb..d88000a 100644
--- a/twill-api/src/main/java/org/apache/twill/api/TwillRunnable.java
+++ b/twill-api/src/main/java/org/apache/twill/api/TwillRunnable.java
@@ -30,7 +30,9 @@ public interface TwillRunnable extends Runnable {
   TwillRunnableSpecification configure();
 
   /**
-   * Called when the container process starts. Executed in container machine.
+   * Called when the container process starts. Executed in container machine. If any exception is thrown from this
+   * method, this runnable won't get retry.
+   *
    * @param context Contains information about the runtime context.
    */
   void initialize(TwillContext context);

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/a77b67cf/twill-yarn/src/main/java/org/apache/twill/internal/ContainerExitCodes.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/ContainerExitCodes.java b/twill-yarn/src/main/java/org/apache/twill/internal/ContainerExitCodes.java
new file mode 100644
index 0000000..22576a9
--- /dev/null
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/ContainerExitCodes.java
@@ -0,0 +1,51 @@
+/*
+ * 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.twill.internal;
+
+/**
+ * Collection of known exit code. Some of the codes are copied from ContainerExitStatus as that class is missing in
+ * older YARN version.
+ */
+public final class ContainerExitCodes {
+
+  public static final int SUCCESS = 0;
+
+  /**
+   * When the container exit when it fails to initilize.
+   */
+  public static final int INIT_FAILED = 10;
+
+  public static final int INVALID = -1000;
+
+  /**
+   * Containers killed by the framework, either due to being released by
+   * the application or being 'lost' due to node failures etc.
+   */
+  public static final int ABORTED = -100;
+
+  /**
+   * When threshold number of the nodemanager-local-directories or
+   * threshold number of the nodemanager-log-directories become bad.
+   */
+  public static final int DISKS_FAILED = -101;
+
+  /**
+   * Containers preempted by the YARN framework.
+   */
+  public static final int PREEMPTED = -102;
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/a77b67cf/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java b/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
index 4831158..740e0e3 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/ServiceMain.java
@@ -73,16 +73,25 @@ public abstract class ServiceMain {
     // Listener for state changes of the service
     ListenableFuture<Service.State> completion = Services.getCompletionFuture(service);
 
-    // Starts the service
-    LOG.info("Starting service {}.", serviceName);
-    Futures.getUnchecked(Services.chainStart(zkClientService, service));
-    LOG.info("Service {} started.", serviceName);
     try {
-      completion.get();
-      LOG.info("Service {} completed.", serviceName);
-    } catch (Throwable t) {
-      LOG.warn("Exception thrown from service {}.", serviceName, t);
-      throw Throwables.propagate(t);
+      try {
+        // Starts the service
+        LOG.info("Starting service {}.", serviceName);
+        Futures.allAsList(Services.chainStart(zkClientService, service).get()).get();
+        LOG.info("Service {} started.", serviceName);
+      } catch (Throwable t) {
+        LOG.error("Exception when starting service {}.", serviceName, t);
+        // Exit with the init fail exit code.
+        System.exit(ContainerExitCodes.INIT_FAILED);
+      }
+
+      try {
+        completion.get();
+        LOG.info("Service {} completed.", serviceName);
+      } catch (Throwable t) {
+        LOG.error("Exception thrown from service {}.", serviceName, t);
+        throw Throwables.propagate(t);
+      }
     } finally {
       ILoggerFactory loggerFactory = LoggerFactory.getILoggerFactory();
       if (loggerFactory instanceof LoggerContext) {

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/a77b67cf/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
index 57c58da..63e3db8 100644
--- a/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
+++ b/twill-yarn/src/main/java/org/apache/twill/internal/appmaster/RunningContainers.java
@@ -34,6 +34,7 @@ import org.apache.twill.api.ResourceReport;
 import org.apache.twill.api.RunId;
 import org.apache.twill.api.ServiceController;
 import org.apache.twill.api.TwillRunResources;
+import org.apache.twill.internal.ContainerExitCodes;
 import org.apache.twill.internal.ContainerInfo;
 import org.apache.twill.internal.DefaultResourceReport;
 import org.apache.twill.internal.DefaultTwillRunResources;
@@ -309,10 +310,13 @@ final class RunningContainers {
         LOG.warn("More than one controller found for container {}", containerId);
       }
 
-      if (exitStatus != 0) {
-        LOG.warn("Container {} exited abnormally with state {}, exit code {}. Re-request the container.",
+      if (exitStatus != ContainerExitCodes.SUCCESS) {
+        LOG.warn("Container {} exited abnormally with state {}, exit code {}.",
                  containerId, state, exitStatus);
-        restartRunnables.add(lookup.keySet().iterator().next());
+        if (shouldRetry(exitStatus)) {
+          LOG.info("Re-request the container {} for exit code {}.", containerId, exitStatus);
+          restartRunnables.add(lookup.keySet().iterator().next());
+        }
       } else {
         LOG.info("Container {} exited normally with state {}", containerId, state);
       }
@@ -333,6 +337,12 @@ final class RunningContainers {
     }
   }
 
+  private boolean shouldRetry(int exitCode) {
+    return exitCode != ContainerExitCodes.SUCCESS
+      && exitCode != ContainerExitCodes.DISKS_FAILED
+      && exitCode != ContainerExitCodes.INIT_FAILED;
+  }
+
   /**
    * Sends a command through the given {@link TwillContainerController} of a runnable. Decrements the count
    * when the sending of command completed. Triggers completion when count reaches zero.

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/a77b67cf/twill-yarn/src/test/java/org/apache/twill/yarn/InitializeFailTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/InitializeFailTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/InitializeFailTestRun.java
new file mode 100644
index 0000000..39813cc
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/InitializeFailTestRun.java
@@ -0,0 +1,68 @@
+/*
+ * 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.twill.yarn;
+
+import org.apache.twill.api.AbstractTwillRunnable;
+import org.apache.twill.api.TwillContext;
+import org.apache.twill.api.TwillController;
+import org.apache.twill.api.TwillRunner;
+import org.apache.twill.api.logging.PrinterLogHandler;
+import org.apache.twill.common.Services;
+import org.junit.Test;
+
+import java.io.PrintWriter;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Test for no retry on runnable initialize failure.
+ */
+public class InitializeFailTestRun extends BaseYarnTest {
+
+  @Test
+  public void testInitFail() throws InterruptedException, ExecutionException, TimeoutException {
+    TwillRunner runner = YarnTestUtils.getTwillRunner();
+    TwillController controller = runner.prepare(new InitFailRunnable())
+                                       .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out)))
+                                       .start();
+
+    Services.getCompletionFuture(controller).get(2, TimeUnit.MINUTES);
+  }
+
+  /**
+   * TwillRunnable class that throws exception in initialize.
+   */
+  public static final class InitFailRunnable extends AbstractTwillRunnable {
+
+    @Override
+    public void initialize(TwillContext context) {
+      throw new IllegalStateException("Fail to init");
+    }
+
+    @Override
+    public void run() {
+      // No-op
+    }
+
+    @Override
+    public void stop() {
+      // No-op
+    }
+  }
+}


[35/50] [abbrv] git commit: #TWILL-51 Added support for Hadoop-2.3.0

Posted by ch...@apache.org.
#TWILL-51 Added support for Hadoop-2.3.0

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: c05a48d0f5b1dd59c76127a3546760c36b89da0d
Parents: a96dbdc
Author: Terence Yim <te...@continuuity.com>
Authored: Tue Mar 4 10:49:45 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Tue Mar 4 15:50:17 2014 -0800

----------------------------------------------------------------------
 pom.xml            | 44 ++++++++++++++++++++++++++++++++++++++++++--
 twill-yarn/pom.xml | 13 +++++++++++++
 2 files changed, 55 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/c05a48d0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0e692f1..8e93850 100644
--- a/pom.xml
+++ b/pom.xml
@@ -182,7 +182,7 @@
         <zookeeper.version>3.4.5</zookeeper.version>
         <junit.version>4.11</junit.version>
         <commons-compress.version>1.5</commons-compress.version>
-        <hadoop.version>[2.0.2-alpha,2.2.0]</hadoop.version>
+        <hadoop.version>[2.0.2-alpha,2.3.0]</hadoop.version>
         <hadoop20.output.dir>target/hadoop20-classes</hadoop20.output.dir>
     </properties>
 
@@ -491,6 +491,47 @@
             <properties>
                 <hadoop.version>2.2.0</hadoop.version>
             </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.codehaus.mojo</groupId>
+                        <artifactId>build-helper-maven-plugin</artifactId>
+                        <version>1.8</version>
+                        <executions>
+                            <execution>
+                                <id>add-source</id>
+                                <phase>generate-sources</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>src/main/hadoop21</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>add-source-2.0</id>
+                                <phase>prepare-package</phase>
+                                <goals>
+                                    <goal>add-source</goal>
+                                </goals>
+                                <configuration>
+                                    <sources>
+                                        <source>src/main/hadoop20</source>
+                                    </sources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+        <profile>
+            <id>hadoop-2.3</id>
+            <properties>
+                <hadoop.version>2.3.0</hadoop.version>
+            </properties>
             <activation>
                 <activeByDefault>true</activeByDefault>
             </activation>
@@ -530,7 +571,6 @@
                 </plugins>
             </build>
         </profile>
-
     </profiles>
 
     <dependencyManagement>

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/c05a48d0/twill-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/twill-yarn/pom.xml b/twill-yarn/pom.xml
index 0c94376..de45335 100644
--- a/twill-yarn/pom.xml
+++ b/twill-yarn/pom.xml
@@ -123,5 +123,18 @@
                 </resources>
             </build>
         </profile>
+        <profile>
+            <id>hadoop-2.3</id>
+            <build>
+                <resources>
+                    <resource>
+                        <directory>${hadoop20.output.dir}</directory>
+                    </resource>
+                    <resource>
+                        <directory>src/main/resources</directory>
+                    </resource>
+                </resources>
+            </build>
+        </profile>
     </profiles>
 </project>


[46/50] [abbrv] git commit: exclude zookeeper.out from rat

Posted by ch...@apache.org.
exclude zookeeper.out from rat


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

Branch: refs/heads/site
Commit: a028289b1c3cc3b52c85dec7948582d2546e5825
Parents: 15f4a7e
Author: anew <an...@continuuity.com>
Authored: Mon Apr 14 14:38:56 2014 -0700
Committer: anew <an...@continuuity.com>
Committed: Mon Apr 14 14:38:56 2014 -0700

----------------------------------------------------------------------
 pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/a028289b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1481d54..2c8f1be 100644
--- a/pom.xml
+++ b/pom.xml
@@ -269,6 +269,7 @@
                             <exclude>target/**/*</exclude>
                             <exclude>**/README</exclude>
                             <exclude>src/test/resources/header.txt</exclude>
+                            <exclude>**/zookeeper.out</exclude>
                         </excludes>
                     </configuration>
                 </plugin>


[19/50] [abbrv] git commit: [PATCH] [TWILL-38] Not setting watch in RewatchOnExpireZKClient if input watcher is null.

Posted by ch...@apache.org.
[PATCH] [TWILL-38] Not setting watch in RewatchOnExpireZKClient if input watcher is null.

Signed-off-by: poorna <po...@continuuity.com>


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

Branch: refs/heads/site
Commit: bb24c023639a76d009f71a4c516b554a9608e41b
Parents: df8e278
Author: poorna <po...@continuuity.com>
Authored: Fri Jan 31 12:59:27 2014 -0800
Committer: poorna <po...@continuuity.com>
Committed: Fri Jan 31 13:06:04 2014 -0800

----------------------------------------------------------------------
 .../twill/internal/zookeeper/RewatchOnExpireZKClient.java   | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/bb24c023/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
----------------------------------------------------------------------
diff --git a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
index 70db61c..bd3bd2d 100644
--- a/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
+++ b/twill-zookeeper/src/main/java/org/apache/twill/internal/zookeeper/RewatchOnExpireZKClient.java
@@ -40,6 +40,9 @@ public final class RewatchOnExpireZKClient extends ForwardingZKClient {
 
   @Override
   public OperationFuture<Stat> exists(String path, Watcher watcher) {
+    if (watcher == null) {
+      return super.exists(path, watcher);
+    }
     final RewatchOnExpireWatcher wrappedWatcher = new RewatchOnExpireWatcher(this, ActionType.EXISTS, path, watcher);
     OperationFuture<Stat> result = super.exists(path, wrappedWatcher);
     Futures.addCallback(result, new FutureCallback<Stat>() {
@@ -58,6 +61,9 @@ public final class RewatchOnExpireZKClient extends ForwardingZKClient {
 
   @Override
   public OperationFuture<NodeChildren> getChildren(String path, Watcher watcher) {
+    if (watcher == null) {
+      return super.getChildren(path, watcher);
+    }
     final RewatchOnExpireWatcher wrappedWatcher = new RewatchOnExpireWatcher(this, ActionType.CHILDREN, path, watcher);
     OperationFuture<NodeChildren> result = super.getChildren(path, wrappedWatcher);
     Futures.addCallback(result, new FutureCallback<NodeChildren>() {
@@ -76,6 +82,9 @@ public final class RewatchOnExpireZKClient extends ForwardingZKClient {
 
   @Override
   public OperationFuture<NodeData> getData(String path, Watcher watcher) {
+    if (watcher == null) {
+      return super.getData(path, watcher);
+    }
     final RewatchOnExpireWatcher wrappedWatcher = new RewatchOnExpireWatcher(this, ActionType.DATA, path, watcher);
     OperationFuture<NodeData> result = super.getData(path, wrappedWatcher);
     Futures.addCallback(result, new FutureCallback<NodeData>() {


[36/50] [abbrv] git commit: (TWILL-52) Improve ZK session expire handling. Avoid killing application if expiration happened by resorting to polling YARN for app status, until the AM is reconnected to ZK.

Posted by ch...@apache.org.
(TWILL-52) Improve ZK session expire handling. Avoid killing application if expiration happened by resorting to polling YARN for app status, until the AM is reconnected to ZK.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 3d3c0e92e8b7455b247202763fc7a2c8a2ff5b71
Parents: c05a48d
Author: Terence Yim <ch...@apache.org>
Authored: Thu Mar 6 02:16:10 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Wed Mar 12 13:02:19 2014 -0700

----------------------------------------------------------------------
 .../internal/AbstractZKServiceController.java   |  85 +++++++----
 .../twill/internal/TwillContainerLauncher.java  |   5 +
 .../twill/internal/ZKServiceDecorator.java      |  28 ++++
 .../apache/twill/internal/ControllerTest.java   |   9 ++
 .../apache/twill/yarn/YarnTwillController.java  | 129 ++++++++++++++--
 .../twill/yarn/YarnTwillRunnerService.java      |   4 +-
 .../apache/twill/yarn/SessionExpireTestRun.java | 147 +++++++++++++++++++
 .../org/apache/twill/yarn/YarnTestSuite.java    |   3 +-
 .../org/apache/twill/yarn/YarnTestUtils.java    |   4 +
 9 files changed, 366 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/3d3c0e92/twill-core/src/main/java/org/apache/twill/internal/AbstractZKServiceController.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/AbstractZKServiceController.java b/twill-core/src/main/java/org/apache/twill/internal/AbstractZKServiceController.java
index a132128..80932ea 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/AbstractZKServiceController.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/AbstractZKServiceController.java
@@ -53,7 +53,7 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
 
   private static final Logger LOG = LoggerFactory.getLogger(AbstractZKServiceController.class);
 
-  private final ZKClient zkClient;
+  protected final ZKClient zkClient;
   private final InstanceNodeDataCallback instanceNodeDataCallback;
   private final StateNodeDataCallback stateNodeDataCallback;
   private final List<ListenableFuture<?>> messageFutures;
@@ -79,6 +79,8 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
 
   @Override
   protected final void startUp() {
+    doStartUp();
+
     // Watch for instance node existence.
     actOnExists(getInstancePath(), new Runnable() {
       @Override
@@ -94,8 +96,6 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
         watchStateNode();
       }
     });
-
-    doStartUp();
   }
 
   @Override
@@ -165,6 +165,12 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
   protected abstract void instanceNodeUpdated(NodeData nodeData);
 
   /**
+   * Called when failed to fetch from live instance node.
+   * @param cause The cause of failure of {@code null} if cause is unknown.
+   */
+  protected abstract void instanceNodeFailed(Throwable cause);
+
+  /**
    * Called when an update on the state node is detected.
    * @param stateNode The update state node data or {@code null} if there is an error when fetching the node data.
    */
@@ -185,6 +191,9 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
     Futures.addCallback(zkClient.exists(path, new Watcher() {
       @Override
       public void process(WatchedEvent event) {
+        if (!shouldProcessZKEvent()) {
+          return;
+        }
         // When node is created, call the action.
         // Other event type would be handled by the action.
         if (event.getType() == Event.EventType.NodeCreated && nodeExists.compareAndSet(false, true)) {
@@ -207,13 +216,14 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
     }, Threads.SAME_THREAD_EXECUTOR);
   }
 
-  private void watchInstanceNode() {
+  protected final void watchInstanceNode() {
+    if (!shouldProcessZKEvent()) {
+      return;
+    }
     Futures.addCallback(zkClient.getData(getInstancePath(), new Watcher() {
       @Override
       public void process(WatchedEvent event) {
-        State state = state();
-        if (state != State.NEW && state != State.STARTING && state != State.RUNNING) {
-          // Ignore ZK node events when it is in stopping sequence.
+        if (!shouldProcessZKEvent()) {
           return;
         }
         switch (event.getType()) {
@@ -221,8 +231,7 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
             watchInstanceNode();
             break;
           case NodeDeleted:
-            // When the ephemeral node goes away, treat the remote service stopped.
-            forceShutDown();
+            instanceNodeFailed(KeeperException.create(KeeperException.Code.NONODE, getInstancePath()));
             break;
           default:
             LOG.info("Ignore ZK event for instance node: {}", event);
@@ -232,12 +241,13 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
   }
 
   private void watchStateNode() {
+    if (!shouldProcessZKEvent()) {
+      return;
+    }
     Futures.addCallback(zkClient.getData(getZKPath("state"), new Watcher() {
       @Override
       public void process(WatchedEvent event) {
-        State state = state();
-        if (state != State.NEW && state != State.STARTING && state != State.RUNNING) {
-          // Ignore ZK node events when it is in stopping sequence.
+        if (!shouldProcessZKEvent()) {
           return;
         }
         switch (event.getType()) {
@@ -245,13 +255,21 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
             watchStateNode();
             break;
           default:
-            LOG.info("Ignore ZK event for state node: {}", event);
+            LOG.debug("Ignore ZK event for state node: {}", event);
         }
       }
     }), stateNodeDataCallback, Threads.SAME_THREAD_EXECUTOR);
   }
 
   /**
+   * Returns true if ZK events needs to be processed.
+   */
+  private boolean shouldProcessZKEvent() {
+    State state = state();
+    return (state == State.NEW || state == State.STARTING || state == State.RUNNING);
+  }
+
+  /**
    * Returns the path prefix for creating sequential message node for the remote service.
    */
   private String getMessagePrefix() {
@@ -261,7 +279,7 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
   /**
    * Returns the zookeeper node path for the ephemeral instance node for this runId.
    */
-  private String getInstancePath() {
+  protected final String getInstancePath() {
     return String.format("/instances/%s", getRunId().getId());
   }
 
@@ -273,17 +291,16 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
 
     @Override
     public void onSuccess(NodeData result) {
-      instanceNodeUpdated(result);
+      if (shouldProcessZKEvent()) {
+        instanceNodeUpdated(result);
+      }
     }
 
     @Override
     public void onFailure(Throwable t) {
       LOG.error("Failed in fetching instance node data.", t);
-      if (t instanceof KeeperException && ((KeeperException) t).code() == KeeperException.Code.NONODE) {
-        // If the node is gone, treat the remote service stopped.
-        forceShutDown();
-      } else {
-        instanceNodeUpdated(null);
+      if (shouldProcessZKEvent()) {
+        instanceNodeFailed(t);
       }
     }
   }
@@ -292,23 +309,27 @@ public abstract class AbstractZKServiceController extends AbstractExecutionServi
 
     @Override
     public void onSuccess(NodeData result) {
-      byte[] data = result.getData();
-      if (data == null) {
-        stateNodeUpdated(null);
-        return;
-      }
-      StateNode stateNode = new GsonBuilder().registerTypeAdapter(StateNode.class, new StateNodeCodec())
-        .registerTypeAdapter(StackTraceElement.class, new StackTraceElementCodec())
-        .create()
-        .fromJson(new String(data, Charsets.UTF_8), StateNode.class);
+      if (shouldProcessZKEvent()) {
+        byte[] data = result.getData();
+        if (data == null) {
+          stateNodeUpdated(null);
+          return;
+        }
+        StateNode stateNode = new GsonBuilder().registerTypeAdapter(StateNode.class, new StateNodeCodec())
+          .registerTypeAdapter(StackTraceElement.class, new StackTraceElementCodec())
+          .create()
+          .fromJson(new String(data, Charsets.UTF_8), StateNode.class);
 
-      stateNodeUpdated(stateNode);
+        stateNodeUpdated(stateNode);
+      }
     }
 
     @Override
     public void onFailure(Throwable t) {
-      LOG.error("Failed in fetching state node data.", t);
-      stateNodeUpdated(null);
+      if (shouldProcessZKEvent()) {
+        LOG.error("Failed in fetching state node data.", t);
+        stateNodeUpdated(null);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/3d3c0e92/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
index 5d2f33c..430e63a 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/TwillContainerLauncher.java
@@ -163,6 +163,11 @@ public final class TwillContainerLauncher {
     }
 
     @Override
+    protected void instanceNodeFailed(Throwable cause) {
+      // No-op
+    }
+
+    @Override
     protected void stateNodeUpdated(StateNode stateNode) {
       // No-op
     }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/3d3c0e92/twill-core/src/main/java/org/apache/twill/internal/ZKServiceDecorator.java
----------------------------------------------------------------------
diff --git a/twill-core/src/main/java/org/apache/twill/internal/ZKServiceDecorator.java b/twill-core/src/main/java/org/apache/twill/internal/ZKServiceDecorator.java
index 7592176..efefe36 100644
--- a/twill-core/src/main/java/org/apache/twill/internal/ZKServiceDecorator.java
+++ b/twill-core/src/main/java/org/apache/twill/internal/ZKServiceDecorator.java
@@ -110,6 +110,7 @@ public final class ZKServiceDecorator extends AbstractService {
   @Override
   protected void doStart() {
     callbackExecutor = Executors.newSingleThreadExecutor(Threads.createDaemonThreadFactory("message-callback"));
+    // Create the live node, if succeeded, start the decorated service, otherwise fail out.
     Futures.addCallback(createLiveNode(), new FutureCallback<String>() {
       @Override
       public void onSuccess(String result) {
@@ -142,6 +143,33 @@ public final class ZKServiceDecorator extends AbstractService {
         notifyFailed(t);
       }
     });
+
+    // Watch for session expiration, recreate the live node if reconnected after expiration.
+    zkClient.addConnectionWatcher(new Watcher() {
+      private boolean expired = false;
+
+      @Override
+      public void process(WatchedEvent event) {
+        if (event.getState() == Event.KeeperState.Expired) {
+          LOG.warn("ZK Session expired for service {} with runId {}.", decoratedService, id.getId());
+          expired = true;
+        } else if (event.getState() == Event.KeeperState.SyncConnected && expired) {
+          LOG.info("Reconnected after expiration for service {} with runId {}", decoratedService, id.getId());
+          expired = false;
+          Futures.addCallback(createLiveNode(), new FutureCallback<String>() {
+            @Override
+            public void onSuccess(String result) {
+              // All good, no-op
+            }
+
+            @Override
+            public void onFailure(Throwable t) {
+              notifyFailed(t);
+            }
+          }, Threads.SAME_THREAD_EXECUTOR);
+        }
+      }
+    });
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/3d3c0e92/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
----------------------------------------------------------------------
diff --git a/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java b/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
index 048e489..4d968bc 100644
--- a/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
+++ b/twill-core/src/test/java/org/apache/twill/internal/ControllerTest.java
@@ -35,6 +35,7 @@ import org.apache.twill.internal.zookeeper.InMemoryZKServer;
 import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.ZKClient;
 import org.apache.twill.zookeeper.ZKClientService;
+import org.apache.zookeeper.KeeperException;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -196,6 +197,14 @@ public class ControllerTest {
       }
 
       @Override
+      protected void instanceNodeFailed(Throwable cause) {
+        // Shutdown if the instance node goes away
+        if (cause instanceof KeeperException.NoNodeException) {
+          forceShutDown();
+        }
+      }
+
+      @Override
       protected void stateNodeUpdated(StateNode stateNode) {
         // No-op
       }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/3d3c0e92/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
index 8e0c6be..5afd679 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillController.java
@@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.commons.lang.time.StopWatch;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.twill.api.ResourceReport;
@@ -37,6 +38,7 @@ import org.apache.twill.internal.state.SystemMessages;
 import org.apache.twill.internal.yarn.YarnApplicationReport;
 import org.apache.twill.zookeeper.NodeData;
 import org.apache.twill.zookeeper.ZKClient;
+import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,7 +46,9 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URL;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 /**
  * A {@link org.apache.twill.api.TwillController} that controllers application running on Hadoop YARN.
@@ -53,20 +57,27 @@ final class YarnTwillController extends AbstractTwillController implements Twill
 
   private static final Logger LOG = LoggerFactory.getLogger(YarnTwillController.class);
 
+  private final String appName;
   private final Callable<ProcessController<YarnApplicationReport>> startUp;
   private ProcessController<YarnApplicationReport> processController;
   private ResourceReportClient resourcesClient;
 
+  // Thread for polling yarn for application status if application got ZK session expire.
+  // Only used by the instanceUpdate/Delete method, which is from serialized call from ZK callback.
+  private Thread statusPollingThread;
+
   /**
    * Creates an instance without any {@link LogHandler}.
    */
-  YarnTwillController(RunId runId, ZKClient zkClient, Callable<ProcessController<YarnApplicationReport>> startUp) {
-    this(runId, zkClient, ImmutableList.<LogHandler>of(), startUp);
+  YarnTwillController(String appName, RunId runId, ZKClient zkClient,
+                      Callable<ProcessController<YarnApplicationReport>> startUp) {
+    this(appName, runId, zkClient, ImmutableList.<LogHandler>of(), startUp);
   }
 
-  YarnTwillController(RunId runId, ZKClient zkClient, Iterable<LogHandler> logHandlers,
+  YarnTwillController(String appName, RunId runId, ZKClient zkClient, Iterable<LogHandler> logHandlers,
                       Callable<ProcessController<YarnApplicationReport>> startUp) {
     super(runId, zkClient, logHandlers);
+    this.appName = appName;
     this.startUp = startUp;
   }
 
@@ -87,7 +98,8 @@ final class YarnTwillController extends AbstractTwillController implements Twill
       processController = startUp.call();
 
       YarnApplicationReport report = processController.getReport();
-      LOG.debug("Application {} submit", report.getApplicationId());
+      ApplicationId appId = report.getApplicationId();
+      LOG.debug("Application {} with id {} submitted", appName, appId);
 
       YarnApplicationState state = report.getYarnApplicationState();
       StopWatch stopWatch = new StopWatch();
@@ -95,18 +107,18 @@ final class YarnTwillController extends AbstractTwillController implements Twill
       stopWatch.split();
       long maxTime = TimeUnit.MILLISECONDS.convert(Constants.APPLICATION_MAX_START_SECONDS, TimeUnit.SECONDS);
 
-      LOG.info("Checking yarn application status");
+      LOG.debug("Checking yarn application status for {} {}", appName, appId);
       while (!hasRun(state) && stopWatch.getSplitTime() < maxTime) {
         report = processController.getReport();
         state = report.getYarnApplicationState();
-        LOG.debug("Yarn application status: {}", state);
+        LOG.debug("Yarn application status for {} {}: {}", appName, appId, state);
         TimeUnit.SECONDS.sleep(1);
         stopWatch.split();
       }
-      LOG.info("Yarn application is in state {}", state);
+      LOG.info("Yarn application {} {} is in state {}", appName, appId, state);
       if (state != YarnApplicationState.RUNNING) {
-        LOG.info("Yarn application is not in running state. Shutting down controller.",
-                 Constants.APPLICATION_MAX_START_SECONDS);
+        LOG.info("Yarn application {} {} is not in running state. Shutting down controller.",
+                 appName, appId, Constants.APPLICATION_MAX_START_SECONDS);
         forceShutDown();
       } else {
         try {
@@ -123,12 +135,15 @@ final class YarnTwillController extends AbstractTwillController implements Twill
   }
 
   @Override
-  protected void doShutDown() {
+  protected synchronized void doShutDown() {
     if (processController == null) {
       LOG.warn("No process controller for application that is not submitted.");
       return;
     }
 
+    // Stop polling if it is running.
+    stopPollStatus();
+
     // Wait for the stop message being processed
     try {
       Uninterruptibles.getUninterruptibly(getStopMessageFuture(),
@@ -148,13 +163,14 @@ final class YarnTwillController extends AbstractTwillController implements Twill
 
       YarnApplicationReport report = processController.getReport();
       FinalApplicationStatus finalStatus = report.getFinalApplicationStatus();
+      ApplicationId appId = report.getApplicationId();
       while (finalStatus == FinalApplicationStatus.UNDEFINED && stopWatch.getSplitTime() < maxTime) {
-        LOG.debug("Yarn application final status for {} {}", report.getApplicationId(), finalStatus);
+        LOG.debug("Yarn application final status for {} {}: {}", appName, appId, finalStatus);
         TimeUnit.SECONDS.sleep(1);
         stopWatch.split();
         finalStatus = processController.getReport().getFinalApplicationStatus();
       }
-      LOG.debug("Yarn application final status is {}", finalStatus);
+      LOG.debug("Yarn application {} {} completed with status {}", appName, appId, finalStatus);
 
       // Application not finished after max stop time, kill the application
       if (finalStatus == FinalApplicationStatus.UNDEFINED) {
@@ -172,7 +188,7 @@ final class YarnTwillController extends AbstractTwillController implements Twill
   public void kill() {
     if (processController != null) {
       YarnApplicationReport report = processController.getReport();
-      LOG.info("Killing application {}", report.getApplicationId());
+      LOG.info("Killing application {} {}", appName, report.getApplicationId());
       processController.cancel();
     } else {
       LOG.warn("No process controller for application that is not submitted.");
@@ -181,7 +197,22 @@ final class YarnTwillController extends AbstractTwillController implements Twill
 
   @Override
   protected void instanceNodeUpdated(NodeData nodeData) {
+  }
+
+  @Override
+  protected void instanceNodeFailed(Throwable cause) {
+    // Resort to polling from Yarn for the application status.
+    if (processController == null) {
+      LOG.warn("No process controller for application that is not submitted.");
+      return;
+    }
+    YarnApplicationReport report = processController.getReport();
+
+    // It happens if the application has ZK session expire or the node is deleted due to application termination.
+    LOG.info("Failed to access application {} {} live node in ZK, resort to polling. Failure reason: {}",
+             appName, report.getApplicationId(), cause == null ? "Unknown" : cause.getMessage());
 
+    startPollStatus(report.getApplicationId());
   }
 
   @Override
@@ -189,6 +220,78 @@ final class YarnTwillController extends AbstractTwillController implements Twill
 
   }
 
+  private synchronized void startPollStatus(ApplicationId appId) {
+    if (statusPollingThread == null) {
+      statusPollingThread = new Thread(createStatusPollingRunnable(),
+                                       String.format("%s-%s-yarn-poller", appName, appId));
+      statusPollingThread.setDaemon(true);
+      statusPollingThread.start();
+    }
+  }
+
+  private synchronized void stopPollStatus() {
+    if (statusPollingThread != null) {
+      statusPollingThread.interrupt();
+      statusPollingThread = null;
+    }
+  }
+
+  private Runnable createStatusPollingRunnable() {
+    return new Runnable() {
+
+      @Override
+      public void run() {
+        YarnApplicationReport report = processController.getReport();
+        ApplicationId appId = report.getApplicationId();
+        boolean shutdown = false;
+        boolean watchInstanceNode = false;
+
+        try {
+          LOG.debug("Polling status from Yarn for {} {}.", appName, appId);
+          while (!Thread.currentThread().isInterrupted()) {
+            if (report.getFinalApplicationStatus() != FinalApplicationStatus.UNDEFINED) {
+              shutdown = true;
+              break;
+            }
+            // Make a sync exists call to instance node and re-watch if the node exists
+            try {
+              // The timeout is arbitrary, as it's just for avoiding block forever
+              Stat stat = zkClient.exists(getInstancePath()).get(5, TimeUnit.SECONDS);
+              if (stat != null) {
+                watchInstanceNode = true;
+                break;
+              }
+            } catch (ExecutionException e) {
+              // Ignore the exception, as any exception won't affect the status polling.
+              LOG.debug("Failed in exists call on ZK path {}.", getInstancePath(), e);
+            } catch (TimeoutException e) {
+              LOG.debug("Timeout in exists call on ZK path {}.", getInstancePath(), e);
+            }
+
+            TimeUnit.SECONDS.sleep(1);
+            report = processController.getReport();
+          }
+        } catch (InterruptedException e) {
+          // OK to ignore.
+          LOG.debug("Status polling thread interrupted for application {} {}", appName, appId);
+        }
+
+        LOG.debug("Stop polling status from Yarn for {} {}.", appName, appId);
+
+        if (shutdown) {
+          LOG.info("Yarn application {} {} completed. Shutting down controller.", appName, appId);
+          forceShutDown();
+        } else if (watchInstanceNode) {
+          LOG.info("Rewatch instance node for {} {} at {}", appName, appId, getInstancePath());
+          synchronized (YarnTwillController.this) {
+            statusPollingThread = null;
+            watchInstanceNode();
+          }
+        }
+      }
+    };
+  }
+
   private boolean hasRun(YarnApplicationState state) {
     switch (state) {
       case RUNNING:

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/3d3c0e92/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
index 9dff24d..72ea58b 100644
--- a/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
+++ b/twill-yarn/src/main/java/org/apache/twill/yarn/YarnTwillRunnerService.java
@@ -245,7 +245,7 @@ public final class YarnTwillRunnerService extends AbstractIdleService implements
       public YarnTwillController create(RunId runId, Iterable<LogHandler> logHandlers,
                                         Callable<ProcessController<YarnApplicationReport>> startUp) {
         ZKClient zkClient = ZKClients.namespace(zkClientService, "/" + appName);
-        YarnTwillController controller = listenController(new YarnTwillController(runId, zkClient,
+        YarnTwillController controller = listenController(new YarnTwillController(appName, runId, zkClient,
                                                                                   logHandlers, startUp));
         synchronized (YarnTwillRunnerService.this) {
           Preconditions.checkArgument(!controllers.contains(appName, runId),
@@ -468,7 +468,7 @@ public final class YarnTwillRunnerService extends AbstractIdleService implements
           if (!controllers.contains(appName, runId)) {
             ZKClient zkClient = ZKClients.namespace(zkClientService, "/" + appName);
             YarnTwillController controller = listenController(
-              new YarnTwillController(runId, zkClient,
+              new YarnTwillController(appName, runId, zkClient,
                                       Callables.returning(yarnAppClient.createProcessController(appId))));
             controllers.put(appName, runId, controller);
             controller.start();

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/3d3c0e92/twill-yarn/src/test/java/org/apache/twill/yarn/SessionExpireTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/SessionExpireTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/SessionExpireTestRun.java
new file mode 100644
index 0000000..ec29061
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/SessionExpireTestRun.java
@@ -0,0 +1,147 @@
+/*
+ * 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.twill.yarn;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.Service;
+import org.apache.twill.api.AbstractTwillRunnable;
+import org.apache.twill.api.TwillController;
+import org.apache.twill.api.TwillRunner;
+import org.apache.twill.api.logging.PrinterLogHandler;
+import org.apache.twill.common.ServiceListenerAdapter;
+import org.apache.twill.common.Services;
+import org.apache.twill.common.Threads;
+import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.server.ConnectionMXBean;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintWriter;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import javax.management.MBeanServer;
+import javax.management.MBeanServerInvocationHandler;
+import javax.management.ObjectName;
+import javax.management.Query;
+import javax.management.QueryExp;
+import javax.management.StringValueExp;
+
+/**
+ * Test for testing ZK session expire from AM container
+ */
+public class SessionExpireTestRun extends BaseYarnTest {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SessionExpireTestRun.class);
+
+  @Test
+  public void testAppSessionExpire() throws InterruptedException, ExecutionException {
+    TwillRunner runner = YarnTestUtils.getTwillRunner();
+    TwillController controller = runner.prepare(new SleepRunnable(600))
+                                       .addLogHandler(new PrinterLogHandler(new PrintWriter(System.out, true)))
+                                       .start();
+
+    final CountDownLatch runLatch = new CountDownLatch(1);
+    controller.addListener(new ServiceListenerAdapter() {
+      @Override
+      public void running() {
+        runLatch.countDown();
+      }
+    }, Threads.SAME_THREAD_EXECUTOR);
+
+    // Wait for application running
+    Assert.assertTrue(runLatch.await(60, TimeUnit.SECONDS));
+
+    // Find the app master ZK session and expire it two times, 10 seconds apart.
+    for (int i = 0; i < 2; i++) {
+      Assert.assertTrue(expireAppMasterZKSession(controller, 10, TimeUnit.SECONDS));
+
+      ListenableFuture<Service.State> completion = Services.getCompletionFuture(controller);
+      try {
+        completion.get(10, TimeUnit.SECONDS);
+        Assert.fail("Unexpected application termination.");
+      } catch (TimeoutException e) {
+        // OK, expected.
+      }
+    }
+
+    controller.stopAndWait();
+  }
+
+  private boolean expireAppMasterZKSession(TwillController controller, long timeout, TimeUnit timeoutUnit) {
+    MBeanServer mbeanServer = MBeanRegistry.getInstance().getPlatformMBeanServer();
+    QueryExp query = Query.isInstanceOf(new StringValueExp(ConnectionMXBean.class.getName()));
+
+    Stopwatch stopwatch = new Stopwatch();
+
+    do {
+      // Find the AM session and expire it
+      Set<ObjectName> connectionBeans = mbeanServer.queryNames(ObjectName.WILDCARD, query);
+      for (ObjectName objectName : connectionBeans) {
+
+        ConnectionMXBean connectionBean = MBeanServerInvocationHandler.newProxyInstance(mbeanServer, objectName,
+                                                                                        ConnectionMXBean.class, false);
+        for (String node : connectionBean.getEphemeralNodes()) {
+          if (node.endsWith("/instances/" + controller.getRunId().getId())) {
+            // This is the AM, expire the session.
+            LOG.info("Kill AM session {}", connectionBean.getSessionId());
+            connectionBean.terminateSession();
+            return true;
+          }
+        }
+      }
+    } while (stopwatch.elapsedTime(timeoutUnit) < timeout);
+
+    return false;
+  }
+
+  /**
+   * A TwillRunnable that just keep sleeping.
+   */
+  public static final class SleepRunnable extends AbstractTwillRunnable {
+
+    private Thread runThread;
+
+    public SleepRunnable(long sleepSeconds) {
+      super(ImmutableMap.of("sleepSeconds", Long.toString(sleepSeconds)));
+    }
+
+    @Override
+    public void stop() {
+      if (runThread != null) {
+        runThread.interrupt();
+      }
+    }
+
+    @Override
+    public void run() {
+      try {
+        runThread = Thread.currentThread();
+        TimeUnit.SECONDS.sleep(Long.parseLong(getArgument("sleepSeconds")));
+      } catch (InterruptedException e) {
+        // Ignore
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/3d3c0e92/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
index bed613a..56fb6a5 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestSuite.java
@@ -32,7 +32,8 @@ import org.junit.runners.Suite;
                       LocalFileTestRun.class,
                       FailureRestartTestRun.class,
                       ProvisionTimeoutTestRun.class,
-                      LogHandlerTestRun.class
+                      LogHandlerTestRun.class,
+                      SessionExpireTestRun.class
                     })
 public final class YarnTestSuite {
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/3d3c0e92/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestUtils.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestUtils.java b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestUtils.java
index 079db56..909c366 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestUtils.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/YarnTestUtils.java
@@ -110,6 +110,10 @@ public final class YarnTestUtils {
     return runnerService;
   }
 
+  public static final InMemoryZKServer getZkServer() {
+    return zkServer;
+  }
+
   /**
    * Creates an unstarted instance of {@link org.apache.twill.api.TwillRunnerService}.
    */


[03/50] [abbrv] git commit: Added README file for simple Getting Started guide and export control statement.

Posted by ch...@apache.org.
Added README file for simple Getting Started guide and export control statement.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 15de6ae079365939b53d0ba3f45e69f9046b04b2
Parents: 9fe7b70
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Jan 20 15:42:17 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Tue Jan 21 14:32:58 2014 -0800

----------------------------------------------------------------------
 README | 44 ++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 44 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/15de6ae0/README
----------------------------------------------------------------------
diff --git a/README b/README
new file mode 100644
index 0000000..5f64cdc
--- /dev/null
+++ b/README
@@ -0,0 +1,44 @@
+What is Apache Twill?
+---------------------
+Twill is an abstraction over Apache Hadoop® YARN that reduces the complexity 
+of developing distributed applications, allowing developers to focus more on 
+their business logic. Twill allows you to use YARN’s distributed capabilities 
+with a programming model that is similar to running threads.
+
+
+Getting Started
+---------------
+You can build and install the Apache Twill by:
+
+    git clone https://git-wip-us.apache.org/repos/asf/incubator-twill.git
+    cd incubator-twill
+    mvn install
+
+After the maven installation completes, you can include the artifact 
+org.apache.twill:twill-yarn as a dependency on your other projects.
+
+Export Control
+-------------
+This distribution includes cryptographic software. The country in which you 
+currently reside may have restrictions on the import, possession, use, and/or
+re-export to another country, of encryption software. BEFORE using any 
+encryption software, please check your country's laws, regulations and 
+policies concerning the import, possession, or use, and re-export of encryption
+software, to see if this is permitted. See <http://www.wassenaar.org/> for more
+information.
+
+The U.S. Government Department of Commerce, Bureau of Industry and Security 
+(BIS), has classified this software as Export Commodity Control Number (ECCN) 
+5D002.C.1, which includes information security software using or performing 
+cryptographic functions with asymmetric algorithms. The form and manner of this
+Apache Software Foundation distribution makes it eligible for export under the 
+License Exception ENC Technology Software Unrestricted (TSU) exception (see the
+BIS Export Administration Regulations, Section 740.13) for both object code and
+source code.
+
+The following provides more details on the included cryptographic software:
+
+Apache Twill uses the built-in java cryptography libraries for unique ID
+generation. See 
+http://www.oracle.com/us/products/export/export-regulations-345813.html
+for more details on Java's cryptography features.


[08/50] [abbrv] git commit: [TWILL-27] Make test less sensitive to machine run time speed.

Posted by ch...@apache.org.
[TWILL-27] Make test less sensitive to machine run time speed.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: c49b2f678e1e1ee6904f81595d928475fbd77bb0
Parents: 979634c
Author: Terence Yim <te...@continuuity.com>
Authored: Mon Jan 27 12:38:02 2014 -0800
Committer: Terence Yim <te...@continuuity.com>
Committed: Mon Jan 27 13:40:45 2014 -0800

----------------------------------------------------------------------
 .../twill/yarn/FailureRestartTestRun.java       | 33 ++++++------
 .../twill/yarn/ResourceReportTestRun.java       | 54 +++++++++++---------
 2 files changed, 46 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/c49b2f67/twill-yarn/src/test/java/org/apache/twill/yarn/FailureRestartTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/FailureRestartTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/FailureRestartTestRun.java
index 06de991..8bc9571 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/FailureRestartTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/FailureRestartTestRun.java
@@ -17,34 +17,31 @@
  */
 package org.apache.twill.yarn;
 
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.Assert;
-import org.junit.Test;
-
 import com.google.common.base.Charsets;
 import com.google.common.collect.Sets;
 import com.google.common.io.LineReader;
-
 import org.apache.twill.api.Command;
 import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.api.TwillController;
 import org.apache.twill.api.TwillRunner;
 import org.apache.twill.api.logging.PrinterLogHandler;
 import org.apache.twill.discovery.Discoverable;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Set;
 
 /**
  *
  */
-public final class FailureRestartTestRun extends  BaseYarnTest {
+public final class FailureRestartTestRun extends BaseYarnTest {
 
   @Test
   public void testFailureRestart() throws Exception {
@@ -70,10 +67,12 @@ public final class FailureRestartTestRun extends  BaseYarnTest {
     // Kill server with instanceId = 0
     controller.sendCommand(FailureRunnable.class.getSimpleName(), Command.Builder.of("kill0").build());
 
-    // Do a shot sleep, make sure the runnable is killed.
-    TimeUnit.SECONDS.sleep(5);
+    // Make sure the runnable is killed.
+    Assert.assertTrue(YarnTestUtils.waitForSize(discoverables, 1, 60));
 
+    // Wait for the restart
     Assert.assertTrue(YarnTestUtils.waitForSize(discoverables, 2, 60));
+
     // Make sure we see the right instance IDs
     Assert.assertEquals(Sets.newHashSet(0, 1), getInstances(discoverables));
 

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/c49b2f67/twill-yarn/src/test/java/org/apache/twill/yarn/ResourceReportTestRun.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/yarn/ResourceReportTestRun.java b/twill-yarn/src/test/java/org/apache/twill/yarn/ResourceReportTestRun.java
index 795caed..71bb607 100644
--- a/twill-yarn/src/test/java/org/apache/twill/yarn/ResourceReportTestRun.java
+++ b/twill-yarn/src/test/java/org/apache/twill/yarn/ResourceReportTestRun.java
@@ -17,28 +17,9 @@
  */
 package org.apache.twill.yarn;
 
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
-import java.net.Socket;
-import java.net.URISyntaxException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
 import com.google.common.base.Charsets;
 import com.google.common.collect.Maps;
 import com.google.common.io.LineReader;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.twill.api.ResourceReport;
 import org.apache.twill.api.ResourceSpecification;
 import org.apache.twill.api.TwillApplication;
@@ -51,12 +32,29 @@ import org.apache.twill.common.ServiceListenerAdapter;
 import org.apache.twill.common.Threads;
 import org.apache.twill.discovery.Discoverable;
 import org.apache.twill.internal.EnvKeys;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.net.Socket;
+import java.net.URISyntaxException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 /**
  * Using echo server to test resource reports.
  * This test is executed by {@link org.apache.twill.yarn.YarnTestUtils}.
  */
-public final class ResourceReportTestRun {
+public final class ResourceReportTestRun extends BaseYarnTest {
 
   private static final Logger LOG = LoggerFactory.getLogger(ResourceReportTestRun.class);
 
@@ -177,10 +175,18 @@ public final class ResourceReportTestRun {
     }
 
     // takes some time for app master to find out the container completed...
-    TimeUnit.SECONDS.sleep(5);
-    // check that we have 1 runnable, not 2.
-    report = controller.getResourceReport();
-    Assert.assertEquals(1, report.getRunnableResources("BuggyServer").size());
+    int count = 0;
+    while (count < 20) {
+      report = controller.getResourceReport();
+      // check that we have 1 runnable, not 2.
+      if (report.getRunnableResources("BuggyServer").size() == 1) {
+        break;
+      }
+      LOG.info("Wait for BuggyServer to have 1 instance left. Trial {}.", count);
+      count++;
+      TimeUnit.SECONDS.sleep(1);
+    }
+    Assert.assertTrue("Still has 2 contains running after 20 seconds", count < 20);
 
     controller.stop().get(30, TimeUnit.SECONDS);
     // Sleep a bit before exiting.


[39/50] [abbrv] git commit: (TWILL-62) Enhance Location API to add support for listing directory content and also check if a location is directory.

Posted by ch...@apache.org.
(TWILL-62) Enhance Location API to add support for listing directory content and also check if a location is directory.

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 6149ea21178b332b8e28e6c4ed9ffc4b42334ade
Parents: d4a352d
Author: Terence Yim <te...@continuuity.com>
Authored: Fri Mar 21 16:33:53 2014 -0700
Committer: Terence Yim <te...@continuuity.com>
Committed: Fri Mar 21 18:05:31 2014 -0700

----------------------------------------------------------------------
 .../apache/twill/filesystem/LocalLocation.java  |  41 +++++++
 .../org/apache/twill/filesystem/Location.java   |  16 +++
 .../twill/filesystem/LocalLocationTest.java     |  64 -----------
 .../apache/twill/filesystem/HDFSLocation.java   |  41 +++++++
 .../twill/filesystem/HDFSLocationTest.java      |  52 +++++++++
 .../twill/filesystem/LocalLocationTest.java     |  40 +++++++
 .../twill/filesystem/LocationTestBase.java      | 111 +++++++++++++++++++
 7 files changed, 301 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-common/src/main/java/org/apache/twill/filesystem/LocalLocation.java
----------------------------------------------------------------------
diff --git a/twill-common/src/main/java/org/apache/twill/filesystem/LocalLocation.java b/twill-common/src/main/java/org/apache/twill/filesystem/LocalLocation.java
index d107eac..c4e3dbf 100644
--- a/twill-common/src/main/java/org/apache/twill/filesystem/LocalLocation.java
+++ b/twill-common/src/main/java/org/apache/twill/filesystem/LocalLocation.java
@@ -17,10 +17,13 @@
  */
 package org.apache.twill.filesystem;
 
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -28,6 +31,7 @@ import java.io.OutputStream;
 import java.net.URI;
 import java.util.Collections;
 import java.util.Deque;
+import java.util.List;
 import java.util.UUID;
 
 /**
@@ -202,4 +206,41 @@ final class LocalLocation implements Location {
   public long lastModified() {
     return file.lastModified();
   }
+
+  @Override
+  public boolean isDirectory() throws IOException {
+    return file.isDirectory();
+  }
+
+  @Override
+  public List<Location> list() throws IOException {
+    File[] files = file.listFiles();
+    ImmutableList.Builder<Location> result = ImmutableList.builder();
+    if (files != null) {
+      for (File file : files) {
+        result.add(new LocalLocation(file));
+      }
+    } else if (!file.exists()) {
+      throw new FileNotFoundException("File " + file + " does not exist.");
+    }
+    return result.build();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    LocalLocation other = (LocalLocation) o;
+    return Objects.equal(file, other.file);
+  }
+
+  @Override
+  public int hashCode() {
+    return file.hashCode();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
----------------------------------------------------------------------
diff --git a/twill-common/src/main/java/org/apache/twill/filesystem/Location.java b/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
index a2d1f9a..db3a8e8 100644
--- a/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
+++ b/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.URI;
+import java.util.List;
 import javax.annotation.Nullable;
 
 /**
@@ -151,4 +152,19 @@ public interface Location {
    * @return Last modified time of file.
    */
   long lastModified() throws IOException;
+
+  /**
+   * Checks if this location represents a directory.
+   *
+   * @return {@code true} if it is a directory, {@code false} otherwise.
+   */
+  boolean isDirectory() throws IOException;
+
+  /**
+   * List the locations under this location.
+   *
+   * @return Immutable List of locations under this location.
+   *         An empty list is returned if this location is not a directory.
+   */
+  List<Location> list() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-common/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
----------------------------------------------------------------------
diff --git a/twill-common/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java b/twill-common/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
deleted file mode 100644
index 198f77f..0000000
--- a/twill-common/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
+++ /dev/null
@@ -1,64 +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.twill.filesystem;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-
-/**
- *
- */
-public class LocalLocationTest {
-
-  @Test
-  public void testDelete() throws IOException {
-    LocationFactory factory = new LocalLocationFactory(new File(System.getProperty("java.io.tmpdir")));
-
-    Location base = factory.create("test").getTempFile(".tmp");
-    Assert.assertTrue(base.mkdirs());
-
-    Assert.assertTrue(base.append("test1").getTempFile(".tmp").createNew());
-    Assert.assertTrue(base.append("test2").getTempFile(".tmp").createNew());
-
-    Location subDir = base.append("test3");
-    Assert.assertTrue(subDir.mkdirs());
-
-    Assert.assertTrue(subDir.append("test4").getTempFile(".tmp").createNew());
-    Assert.assertTrue(subDir.append("test5").getTempFile(".tmp").createNew());
-
-    Assert.assertTrue(base.delete(true));
-    Assert.assertFalse(base.exists());
-  }
-
-  @Test
-  public void testHelper() {
-    LocationFactory factory = LocationFactories.namespace(
-                                new LocalLocationFactory(new File(System.getProperty("java.io.tmpdir"))),
-                                "testhelper");
-
-    Location location = factory.create("test");
-    Assert.assertTrue(location.toURI().getPath().endsWith("testhelper/test"));
-
-    location = factory.create(URI.create("test2"));
-    Assert.assertTrue(location.toURI().getPath().endsWith("testhelper/test2"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java b/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java
index b0eeb43..59a371c 100644
--- a/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java
+++ b/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java
@@ -17,7 +17,10 @@
  */
 package org.apache.twill.filesystem;
 
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
@@ -28,6 +31,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.URI;
+import java.util.List;
 import java.util.UUID;
 
 /**
@@ -190,4 +194,41 @@ final class HDFSLocation implements Location {
   public long lastModified() throws IOException {
     return fs.getFileStatus(path).getModificationTime();
   }
+
+  @Override
+  public boolean isDirectory() throws IOException {
+    return fs.isDirectory(path);
+  }
+
+  @Override
+  public List<Location> list() throws IOException {
+    FileStatus[] statuses = fs.listStatus(path);
+    ImmutableList.Builder<Location> result = ImmutableList.builder();
+    if (statuses != null) {
+      for (FileStatus status : statuses) {
+        if (!Objects.equal(path, status.getPath())) {
+          result.add(new HDFSLocation(fs, status.getPath()));
+        }
+      }
+    }
+    return result.build();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    HDFSLocation other = (HDFSLocation) o;
+    return Objects.equal(path, other.path);
+  }
+
+  @Override
+  public int hashCode() {
+    return path.hashCode();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java b/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java
new file mode 100644
index 0000000..20f7403
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.twill.filesystem;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ *
+ */
+public class HDFSLocationTest extends LocationTestBase {
+
+  private static MiniDFSCluster dfsCluster;
+  private static LocationFactory locationFactory;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    Configuration conf = new Configuration();
+    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, tmpFolder.newFolder().getAbsolutePath());
+    dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    locationFactory = new HDFSLocationFactory(dfsCluster.getFileSystem());
+  }
+
+  @AfterClass
+  public static void finish() {
+    dfsCluster.shutdown();
+  }
+
+  @Override
+  protected LocationFactory getLocationFactory() {
+    return locationFactory;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
new file mode 100644
index 0000000..3f6d931
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.twill.filesystem;
+
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ *
+ */
+public class LocalLocationTest extends LocationTestBase {
+
+  private static LocationFactory locationFactory;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    locationFactory = new LocalLocationFactory(tmpFolder.newFolder());
+  }
+
+  @Override
+  protected LocationFactory getLocationFactory() {
+    return locationFactory;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java
new file mode 100644
index 0000000..ee591e7
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java
@@ -0,0 +1,111 @@
+/*
+ * 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.twill.filesystem;
+
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+
+/**
+ * Base class for defining {@link Location} and {@link LocationFactory} tests.
+ */
+public abstract class LocationTestBase {
+
+  @ClassRule
+  public static TemporaryFolder tmpFolder = new TemporaryFolder();
+
+
+  @Test
+  public void testDelete() throws IOException {
+    LocationFactory factory = getLocationFactory();
+
+    Location base = factory.create("test").getTempFile(".tmp");
+    Assert.assertTrue(base.mkdirs());
+
+    Assert.assertTrue(base.append("test1").getTempFile(".tmp").createNew());
+    Assert.assertTrue(base.append("test2").getTempFile(".tmp").createNew());
+
+    Location subDir = base.append("test3");
+    Assert.assertTrue(subDir.mkdirs());
+
+    Assert.assertTrue(subDir.append("test4").getTempFile(".tmp").createNew());
+    Assert.assertTrue(subDir.append("test5").getTempFile(".tmp").createNew());
+
+    Assert.assertTrue(base.delete(true));
+    Assert.assertFalse(base.exists());
+  }
+
+  @Test
+  public void testHelper() {
+    LocationFactory factory = LocationFactories.namespace(getLocationFactory(), "testhelper");
+
+    Location location = factory.create("test");
+    Assert.assertTrue(location.toURI().getPath().endsWith("testhelper/test"));
+
+    location = factory.create(URI.create("test2"));
+    Assert.assertTrue(location.toURI().getPath().endsWith("testhelper/test2"));
+  }
+
+  @Test
+  public void testList() throws IOException {
+    LocationFactory factory = getLocationFactory();
+
+    Location dir = factory.create("dir");
+
+    // Check and create the directory
+    Assert.assertFalse(dir.isDirectory());
+    Assert.assertTrue(dir.mkdirs());
+    Assert.assertTrue(dir.isDirectory());
+
+    // Should have nothing inside
+    Assert.assertTrue(dir.list().isEmpty());
+
+    // Check and create a file inside the directory
+    Location file = dir.append("file");
+    Assert.assertFalse(file.isDirectory());
+    Assert.assertTrue(file.createNew());
+    Assert.assertFalse(file.isDirectory());
+
+    // List on file should gives empty list
+    Assert.assertTrue(file.list().isEmpty());
+
+    // List on directory should gives the file inside
+    List<Location> listing = dir.list();
+    Assert.assertEquals(1, listing.size());
+    Assert.assertEquals(file, listing.get(0));
+
+    // After deleting the file inside the directory, list on directory should be empty again.
+    file.delete();
+    Assert.assertTrue(dir.list().isEmpty());
+
+    // List on a non-exist location would throw exception
+    try {
+      file.list();
+      Assert.fail("List should fail on non-exist location.");
+    } catch (IOException e) {
+      // Expected
+    }
+  }
+
+  protected abstract LocationFactory getLocationFactory();
+}


[24/50] [abbrv] git commit: upgrade snappy-java to version 1.0.5 fixes #TWILL-39

Posted by ch...@apache.org.
upgrade snappy-java to version 1.0.5 fixes #TWILL-39

Signed-off-by: Terence Yim <te...@continuuity.com>


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

Branch: refs/heads/site
Commit: 652b91c20226f9daf59c09c87cdd9a8bb2ffc203
Parents: 64a0140
Author: Fabian Murariu <mu...@gmail.com>
Authored: Tue Feb 11 11:22:11 2014 +0200
Committer: Terence Yim <te...@continuuity.com>
Committed: Tue Feb 11 12:55:41 2014 -0800

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/652b91c2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b1c4bfe..4572d08 100644
--- a/pom.xml
+++ b/pom.xml
@@ -572,7 +572,7 @@
             <dependency>
                 <groupId>org.xerial.snappy</groupId>
                 <artifactId>snappy-java</artifactId>
-                <version>1.0.4.1</version>
+                <version>1.0.5</version>
             </dependency>
             <dependency>
                 <groupId>org.slf4j</groupId>