Browse Source

YARN-1780. Improved logging in the Timeline client and server. Contributed by Zhijie Shen.
svn merge --ignore-ancestry -c 1575141 ../../trunk/


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.4@1575147 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 11 years ago
parent
commit
bd5901ee98

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -220,6 +220,9 @@ Release 2.4.0 - UNRELEASED
     YARN-1761. Modified RMAdmin CLI to check whether HA is enabled or not before
     YARN-1761. Modified RMAdmin CLI to check whether HA is enabled or not before
     it executes any of the HA admin related commands. (Xuan Gong via vinodkv)
     it executes any of the HA admin related commands. (Xuan Gong via vinodkv)
 
 
+    YARN-1780. Improved logging in the Timeline client and server. (Zhijie Shen
+    via vinodkv)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

+ 14 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java

@@ -75,6 +75,7 @@ public class TimelineClientImpl extends TimelineClient {
           YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
           YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
           YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS), RESOURCE_URI_STR));
           YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS), RESOURCE_URI_STR));
     }
     }
+    LOG.info("Timeline service address: " + resURI);
     super.serviceInit(conf);
     super.serviceInit(conf);
   }
   }
 
 
@@ -83,12 +84,22 @@ public class TimelineClientImpl extends TimelineClient {
       TimelineEntity... entities) throws IOException, YarnException {
       TimelineEntity... entities) throws IOException, YarnException {
     TimelineEntities entitiesContainer = new TimelineEntities();
     TimelineEntities entitiesContainer = new TimelineEntities();
     entitiesContainer.addEntities(Arrays.asList(entities));
     entitiesContainer.addEntities(Arrays.asList(entities));
-    ClientResponse resp = doPostingEntities(entitiesContainer);
-    if (resp.getClientResponseStatus() != ClientResponse.Status.OK) {
+    ClientResponse resp;
+    try {
+      resp = doPostingEntities(entitiesContainer);
+    } catch (RuntimeException re) {
+      // runtime exception is expected if the client cannot connect the server
+      String msg =
+          "Failed to get the response from the timeline server.";
+      LOG.error(msg, re);
+      throw re;
+    }
+    if (resp == null ||
+        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
       String msg =
       String msg =
           "Failed to get the response from the timeline server.";
           "Failed to get the response from the timeline server.";
       LOG.error(msg);
       LOG.error(msg);
-      if (LOG.isDebugEnabled()) {
+      if (LOG.isDebugEnabled() && resp != null) {
         String output = resp.getEntity(String.class);
         String output = resp.getEntity(String.class);
         LOG.debug("HTTP error code: " + resp.getStatus()
         LOG.debug("HTTP error code: " + resp.getStatus()
             + " Server response : \n" + output);
             + " Server response : \n" + output);

+ 26 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java

@@ -20,9 +20,13 @@ package org.apache.hadoop.yarn.client.api.impl;
 
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
+
+import java.net.ConnectException;
+
 import junit.framework.Assert;
 import junit.framework.Assert;
 
 
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
@@ -36,6 +40,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import com.sun.jersey.api.client.ClientHandlerException;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse;
 
 
 public class TestTimelineClient {
 public class TestTimelineClient {
@@ -56,7 +61,7 @@ public class TestTimelineClient {
 
 
   @Test
   @Test
   public void testPostEntities() throws Exception {
   public void testPostEntities() throws Exception {
-    mockClientResponse(ClientResponse.Status.OK, false);
+    mockClientResponse(ClientResponse.Status.OK, false, false);
     try {
     try {
       TimelinePutResponse response = client.putEntities(generateEntity());
       TimelinePutResponse response = client.putEntities(generateEntity());
       Assert.assertEquals(0, response.getErrors().size());
       Assert.assertEquals(0, response.getErrors().size());
@@ -67,7 +72,7 @@ public class TestTimelineClient {
 
 
   @Test
   @Test
   public void testPostEntitiesWithError() throws Exception {
   public void testPostEntitiesWithError() throws Exception {
-    mockClientResponse(ClientResponse.Status.OK, true);
+    mockClientResponse(ClientResponse.Status.OK, true, false);
     try {
     try {
       TimelinePutResponse response = client.putEntities(generateEntity());
       TimelinePutResponse response = client.putEntities(generateEntity());
       Assert.assertEquals(1, response.getErrors().size());
       Assert.assertEquals(1, response.getErrors().size());
@@ -84,7 +89,8 @@ public class TestTimelineClient {
 
 
   @Test
   @Test
   public void testPostEntitiesNoResponse() throws Exception {
   public void testPostEntitiesNoResponse() throws Exception {
-    mockClientResponse(ClientResponse.Status.INTERNAL_SERVER_ERROR, false);
+    mockClientResponse(
+        ClientResponse.Status.INTERNAL_SERVER_ERROR, false, false);
     try {
     try {
       client.putEntities(generateEntity());
       client.putEntities(generateEntity());
       Assert.fail("Exception is expected");
       Assert.fail("Exception is expected");
@@ -94,9 +100,25 @@ public class TestTimelineClient {
     }
     }
   }
   }
 
 
+  @Test
+  public void testPostEntitiesConnectionRefused() throws Exception {
+    mockClientResponse(null, false, true);
+    try {
+      client.putEntities(generateEntity());
+      Assert.fail("RuntimeException is expected");
+    } catch (RuntimeException re) {
+      Assert.assertTrue(re instanceof ClientHandlerException);
+    }
+  }
+
   private ClientResponse mockClientResponse(ClientResponse.Status status,
   private ClientResponse mockClientResponse(ClientResponse.Status status,
-      boolean hasError) {
+      boolean hasError, boolean hasRuntimeError) {
     ClientResponse response = mock(ClientResponse.class);
     ClientResponse response = mock(ClientResponse.class);
+    if (hasRuntimeError) {
+      doThrow(new ClientHandlerException(new ConnectException())).when(client)
+          .doPostingEntities(any(TimelineEntities.class));
+      return response;
+    }
     doReturn(response).when(client)
     doReturn(response).when(client)
         .doPostingEntities(any(TimelineEntities.class));
         .doPostingEntities(any(TimelineEntities.class));
     when(response.getClientResponseStatus()).thenReturn(status);
     when(response.getClientResponseStatus()).thenReturn(status);

+ 16 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TimelineWebServices.java

@@ -18,6 +18,8 @@
 
 
 package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
 package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
 
 
+import static org.apache.hadoop.yarn.util.StringHelper.CSV_JOINER;
+
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
@@ -54,10 +56,12 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.EntityIdentifier;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.GenericObjectMapper;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.GenericObjectMapper;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.NameValuePair;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.NameValuePair;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineReader.Field;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 
 
 import com.google.inject.Inject;
 import com.google.inject.Inject;
@@ -245,6 +249,17 @@ public class TimelineWebServices {
       return new TimelinePutResponse();
       return new TimelinePutResponse();
     }
     }
     try {
     try {
+      List<EntityIdentifier> entityIDs = new ArrayList<EntityIdentifier>();
+      for (TimelineEntity entity : entities.getEntities()) {
+        EntityIdentifier entityID =
+            new EntityIdentifier(entity.getEntityId(), entity.getEntityType());
+        entityIDs.add(entityID);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Storing the entity " + entityID + ", JSON-style content: "
+              + TimelineUtils.dumpTimelineRecordtoJSON(entity));
+        }
+      }
+      LOG.info("Storing entities: " + CSV_JOINER.join(entityIDs));
       return store.put(entities);
       return store.put(entities);
     } catch (IOException e) {
     } catch (IOException e) {
       LOG.error("Error putting entities", e);
       LOG.error("Error putting entities", e);