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;
* });
* ...
* ...
- * 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">-->
+ <!--<!– Checks that each Java package has a Javadoc file used for commenting.-->
+ <!--Only allows a package-info.java, not package.html. –>-->
+ <!--<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;
* @Override
* public String getName() {
* return 'service-name';
- * }
+ * }
*
* @Override
* public InetSocketAddress getSocketAddress() {
* return new InetSocketAddress(hostname, port);
- * }
- * });
+ * }
+ * });
* ...
* ...
* 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>