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/02/14 09:21:19 UTC

git commit: (TWILL-45) Included Throwable information in LogEntry

Updated Branches:
  refs/heads/master 146740b12 -> e5c62e5cd


(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/master
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