You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by st...@apache.org on 2014/05/16 18:41:36 UTC

svn commit: r1595261 - in /incubator/slider/trunk: slider-core/src/main/java/org/apache/slider/core/registry/info/ slider-core/src/main/java/org/apache/slider/providers/ slider-core/src/main/java/org/apache/slider/providers/agent/ slider-core/src/main/...

Author: stevel
Date: Fri May 16 16:41:36 2014
New Revision: 1595261

URL: http://svn.apache.org/r1595261
Log:
SLIDER-35 providers to publish registry information - all providers enum their endpoints and list what there is

Added:
    incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/DemoHBaseCluster.groovy
Modified:
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/RegisteredEndpoint.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryRestResources.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java
    incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java
    incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy
    incubator/slider/trunk/slider-providers/accumulo/slider-accumulo-provider/src/main/java/org/apache/slider/providers/accumulo/AccumuloProviderService.java
    incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestFunctionalHBaseCluster.groovy
    incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/RegisteredEndpoint.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/RegisteredEndpoint.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/RegisteredEndpoint.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/core/registry/info/RegisteredEndpoint.java Fri May 16 16:41:36 2014
@@ -19,6 +19,7 @@
 package org.apache.slider.core.registry.info;
 
 import org.apache.slider.core.exceptions.SliderException;
+import org.codehaus.jackson.annotate.JsonIgnore;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
@@ -174,6 +175,7 @@ public class RegisteredEndpoint {
     return sb.toString();
   }
 
+  @JsonIgnore
   public boolean isHttpProtocol() {
     return PROTOCOL_HTTP.equals(protocol) || PROTOCOL_HTTPS.equals(protocol);
   }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java Fri May 16 16:41:36 2014
@@ -28,9 +28,16 @@ import org.apache.slider.core.conf.Aggre
 import org.apache.slider.core.exceptions.BadCommandArgumentsException;
 import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.core.main.ExitCodeProvider;
+import org.apache.slider.core.registry.info.RegisteredEndpoint;
 import org.apache.slider.core.registry.info.ServiceInstanceData;
+import org.apache.slider.providers.agent.AgentProviderService;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
 import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
+import org.apache.slider.server.appmaster.web.rest.agent.HeartBeat;
+import org.apache.slider.server.appmaster.web.rest.agent.HeartBeatResponse;
+import org.apache.slider.server.appmaster.web.rest.agent.Register;
+import org.apache.slider.server.appmaster.web.rest.agent.RegistrationResponse;
+import org.apache.slider.server.appmaster.web.rest.agent.RegistrationStatus;
 import org.apache.slider.server.services.curator.RegistryBinderService;
 import org.apache.slider.server.services.registry.RegistryViewForProviders;
 import org.apache.slider.server.services.utility.ForkedProcessService;
@@ -41,10 +48,12 @@ import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -256,7 +265,6 @@ public abstract class AbstractProviderSe
     return process;
   }
 
-
   /*
    * Build the provider status, can be empty
    * @return the provider status - map of entries to add to the info section
@@ -266,12 +274,18 @@ public abstract class AbstractProviderSe
     return new HashMap<String, String>();
   }
 
-  /* non-javadoc
-   * @see org.apache.slider.providers.ProviderService#buildMonitorDetails(org.apache.slider.api.ClusterDescription)
+  /*
+  Build the monitor details. The base implementation includes all the external URL endpoints
+  in the external view
    */
   @Override
   public Map<String,URL> buildMonitorDetails(ClusterDescription clusterDesc) {
-    return Collections.emptyMap();
+    Map<String, URL> details = new LinkedHashMap<>();
+
+    // add in all the 
+    buildEndpointDetails(details);
+
+    return details;
   }
   
   protected String getInfoAvoidingNull(ClusterDescription clusterDesc, String key) {
@@ -279,4 +293,27 @@ public abstract class AbstractProviderSe
 
     return null == value ? "N/A" : value;
   }
+
+  @Override
+  public void buildEndpointDetails(Map<String, URL> details) {
+      ServiceInstanceData self = registry.getSelfRegistration();
+    buildEndpointDetails(details, self);
+  }
+
+  public static void buildEndpointDetails(Map<String, URL> details,
+      ServiceInstanceData self) {
+    Map<String, RegisteredEndpoint> endpoints =
+        self.getRegistryView(true).endpoints;
+    for (Map.Entry<String, RegisteredEndpoint> endpoint : endpoints.entrySet()) {
+      RegisteredEndpoint val = endpoint.getValue();
+      if (val.type.equals(RegisteredEndpoint.TYPE_URL)) {
+        try {
+          URL url = new URL(val.value);
+          details.put(val.description, url);
+        } catch (MalformedURLException e) {
+          log.warn("Failed to create URL from {} : {} ",val.value, e);
+        }
+      }
+    }
+  }
 }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java Fri May 16 16:41:36 2014
@@ -148,4 +148,10 @@ public interface ProviderService extends
    * @return  the interface if available, null otherwise.
    */
   AgentRestOperations getAgentRestOperations();
+
+  /**
+   * Build up the endpoint details for this service
+   * @param details
+   */
+  void buildEndpointDetails(Map<String, URL> details);
 }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java Fri May 16 16:41:36 2014
@@ -41,6 +41,8 @@ import org.apache.slider.core.exceptions
 import org.apache.slider.core.launch.CommandLineBuilder;
 import org.apache.slider.core.launch.ContainerLauncher;
 import org.apache.slider.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.core.registry.info.RegisteredEndpoint;
+import org.apache.slider.core.registry.info.ServiceInstanceData;
 import org.apache.slider.providers.AbstractProviderService;
 import org.apache.slider.providers.ProviderCore;
 import org.apache.slider.providers.ProviderRole;
@@ -70,6 +72,7 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
+import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URL;
 import java.util.ArrayList;
@@ -98,7 +101,7 @@ public class AgentProviderService extend
   private static final String CONTAINER_ID = "container_id";
   private static final String GLOBAL_CONFIG_TAG = "global";
   private AgentClientProvider clientProvider;
-  private Map<String, ComponentInstanceState> componentStatuses = new HashMap<String, ComponentInstanceState>();
+  private Map<String, ComponentInstanceState> componentStatuses = new HashMap<>();
   private AtomicInteger taskId = new AtomicInteger(0);
   private Metainfo metainfo = null;
 
@@ -307,7 +310,7 @@ public class AgentProviderService extend
    * @return the provider status - map of entries to add to the info section
    */
   public Map<String, String> buildProviderStatus() {
-    Map<String, String> stats = new HashMap<String, String>();
+    Map<String, String> stats = new HashMap<>();
     return stats;
   }
 
@@ -430,7 +433,7 @@ public class AgentProviderService extend
 
   protected void processReturnedStatus(HeartBeat heartBeat, ComponentInstanceState componentStatus) {
     List<ComponentStatus> statuses = heartBeat.getComponentStatus();
-    if (statuses != null && statuses.size() > 0) {
+    if (statuses != null && !statuses.isEmpty()) {
       log.info("Processing {} status reports.", statuses.size());
       for (ComponentStatus status : statuses) {
         log.info("Status report: " + status.toString());
@@ -442,7 +445,7 @@ public class AgentProviderService extend
 
           Service service = getMetainfo().getServices().get(0);
           List<ExportGroup> exportGroups = service.getExportGroups();
-          if (exportGroups != null && exportGroups.size() > 0) {
+          if (exportGroups != null && !exportGroups.isEmpty()) {
 
             String configKeyFormat = "${site.%s.%s}";
             String hostKeyFormat = "${%s_HOST}";
@@ -464,7 +467,7 @@ public class AgentProviderService extend
 
             for (ExportGroup exportGroup : exportGroups) {
               List<Export> exports = exportGroup.getExports();
-              if (exports != null && exports.size() > 0) {
+              if (exports != null && !exports.isEmpty()) {
                 String exportGroupName = exportGroup.getName();
                 Map<String, String> map = new HashMap<>();
                 for (Export export : exports) {
@@ -546,7 +549,7 @@ public class AgentProviderService extend
     cmd.setServiceName(clusterName);
     cmd.setComponentName(roleName);
     cmd.setRole(roleName);
-    Map<String, String> hostLevelParams = new TreeMap<String, String>();
+    Map<String, String> hostLevelParams = new TreeMap<>();
     hostLevelParams.put(JAVA_HOME, appConf.getGlobalOptions().getMandatoryOption(JAVA_HOME));
     hostLevelParams.put(PACKAGE_LIST, "[{\"type\":\"tarball\",\"name\":\"" +
                                       appConf.getGlobalOptions().getMandatoryOption(
@@ -568,7 +571,7 @@ public class AgentProviderService extend
   }
 
   private Map<String, String> setCommandParameters(String scriptPath, boolean recordConfig) {
-    Map<String, String> cmdParams = new TreeMap<String, String>();
+    Map<String, String> cmdParams = new TreeMap<>();
     cmdParams.put("service_package_folder",
                   "${AGENT_WORK_ROOT}/work/app/definition/package");
     cmdParams.put("script", scriptPath);
@@ -600,7 +603,7 @@ public class AgentProviderService extend
     cmd.setClusterName(clusterName);
     cmd.setRoleCommand(StatusCommand.STATUS_COMMAND);
 
-    Map<String, String> hostLevelParams = new TreeMap<String, String>();
+    Map<String, String> hostLevelParams = new TreeMap<>();
     hostLevelParams.put(JAVA_HOME, appConf.getGlobalOptions().getMandatoryOption(JAVA_HOME));
     hostLevelParams.put(CONTAINER_ID, containerId);
     cmd.setHostLevelParams(hostLevelParams);
@@ -627,7 +630,7 @@ public class AgentProviderService extend
     cmd.setServiceName(clusterName);
     cmd.setClusterName(clusterName);
     cmd.setRoleCommand(StatusCommand.GET_CONFIG_COMMAND);
-    Map<String, String> hostLevelParams = new TreeMap<String, String>();
+    Map<String, String> hostLevelParams = new TreeMap<>();
     hostLevelParams.put(CONTAINER_ID, containerId);
     cmd.setHostLevelParams(hostLevelParams);
 
@@ -653,7 +656,7 @@ public class AgentProviderService extend
     cmd.setServiceName(clusterName);
     cmd.setComponentName(roleName);
     cmd.setRole(roleName);
-    Map<String, String> hostLevelParams = new TreeMap<String, String>();
+    Map<String, String> hostLevelParams = new TreeMap<>();
     hostLevelParams.put(JAVA_HOME, appConf.getGlobalOptions().getMandatoryOption(JAVA_HOME));
     hostLevelParams.put(CONTAINER_ID, containerId);
     cmd.setHostLevelParams(hostLevelParams);
@@ -668,7 +671,7 @@ public class AgentProviderService extend
 
   private Map<String, Map<String, String>> buildCommandConfigurations(ConfTreeOperations appConf) {
 
-    Map<String, Map<String, String>> configurations = new TreeMap<String, Map<String, String>>();
+    Map<String, Map<String, String>> configurations = new TreeMap<>();
     Map<String, String> tokens = getStandardTokenMap(appConf);
 
     List<String> configs = getApplicationConfigurationTypes(appConf);
@@ -683,7 +686,7 @@ public class AgentProviderService extend
   }
 
   private Map<String, String> getStandardTokenMap(ConfTreeOperations appConf) {
-    Map<String, String> tokens = new HashMap<String, String>();
+    Map<String, String> tokens = new HashMap<>();
     String nnuri = appConf.get("site.fs.defaultFS");
     tokens.put("${NN_URI}", nnuri);
     tokens.put("${NN_HOST}", URI.create(nnuri).getHost());
@@ -694,7 +697,7 @@ public class AgentProviderService extend
   private List<String> getApplicationConfigurationTypes(ConfTreeOperations appConf) {
     // for now, reading this from appConf.  In the future, modify this method to
     // process metainfo.xml
-    List<String> configList = new ArrayList<String>();
+    List<String> configList = new ArrayList<>();
     configList.add(GLOBAL_CONFIG_TAG);
 
     String configTypes = appConf.get("config_types");
@@ -703,13 +706,13 @@ public class AgentProviderService extend
     configList.addAll(Arrays.asList(configs));
 
     // remove duplicates.  mostly worried about 'global' being listed
-    return new ArrayList<String>(new HashSet<String>(configList));
+    return new ArrayList<>(new HashSet<>(configList));
   }
 
   private void addNamedConfiguration(String configName, Map<String, String> sourceConfig,
                                      Map<String, Map<String, String>> configurations,
                                      Map<String, String> tokens) {
-    Map<String, String> config = new HashMap<String, String>();
+    Map<String, String> config = new HashMap<>();
     if (configName.equals(GLOBAL_CONFIG_TAG)) {
       addDefaultGlobalConfig(config);
     }
@@ -745,8 +748,7 @@ public class AgentProviderService extend
 
   @Override
   public Map<String, URL> buildMonitorDetails(ClusterDescription clusterDesc) {
-    Map<String, URL> details = new LinkedHashMap<String, URL>();
-    buildEndpointDetails(details);
+    Map<String, URL> details = super.buildMonitorDetails(clusterDesc);
     buildRoleHostDetails(details);
     return details;
   }
@@ -760,23 +762,4 @@ public class AgentProviderService extend
     }
   }
 
-  private void buildEndpointDetails(Map<String, URL> details) {
-    try {
-      List services =
-          registry.listInstancesByType(SliderKeys.APP_TYPE);
-      assert services.size() >= 1;
-      Map payload = (Map) services.get(0);
-      Map<String, Map> endpointMap =
-          (Map<String, Map>) ((Map) payload.get("externalView")).get("endpoints");
-      for (Map.Entry<String, Map> endpoint : endpointMap.entrySet()) {
-        Map<String, String> val = endpoint.getValue();
-        if ("http".equals(val.get("protocol"))) {
-          URL url = new URL(val.get("value"));
-          details.put(val.get("description"), url);
-        }
-      }
-    } catch (IOException e) {
-      log.error("Error creating list of slider URIs", e);
-    }
-  }
 }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java Fri May 16 16:41:36 2014
@@ -760,7 +760,8 @@ public class SliderAppMaster extends Abs
     externalView.endpoints.put(
         CustomRegistryConstants.REGISTRY_REST_API,
       new RegisteredEndpoint(
-        new URL(amWeb, RestPaths.SLIDER_PATH_REGISTRY),
+        new URL(amWeb, RestPaths.SLIDER_PATH_REGISTRY + "/"+
+                       RestPaths.REGISTRY_SERVICE),
         "Registry Web Service" )
     );
 

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java Fri May 16 16:41:36 2014
@@ -99,7 +99,6 @@ public class IndexBlock extends HtmlBloc
     if (null == details) {
       return;
     }
-    
     // Loop over each entry, placing the text in the UL, adding an anchor when the URL is non-null
     for (Entry<String,URL> entry : details.entrySet()) {
       if (null != entry.getValue()) {

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/curator/RegistryBinderService.java Fri May 16 16:41:36 2014
@@ -164,6 +164,12 @@ public class RegistryBinderService<Paylo
   }
 
 
+  /**
+   * List all instance IDs of a service type
+   * @param servicetype service type
+   * @return list of matches
+   * @throws Exception
+   */
   public List<String> instanceIDs(String servicetype) throws Exception {
     Preconditions.checkNotNull(servicetype);
     List<String> instanceIds;
@@ -176,6 +182,22 @@ public class RegistryBinderService<Paylo
     return instanceIds;
   }
 
+  /**
+   * List all service types registered
+   * @return
+   * @throws Exception
+   */
+  public List<String> serviceTypes() throws Exception {
+    List<String> types;
+    try {
+      types =
+        getCurator().getChildren().forPath(getBasePath());
+    } catch (KeeperException.NoNodeException e) {
+      types = Lists.newArrayList();
+    }
+    return types;
+  }
+
 
   /**
    * Return a service instance POJO
@@ -185,8 +207,8 @@ public class RegistryBinderService<Paylo
    * @return the instance or <code>null</code> if not found
    * @throws Exception errors
    */
-  public CuratorServiceInstance<Payload> queryForInstance(String servicetype, String id) throws
-                                                                         Exception {
+  public CuratorServiceInstance<Payload> queryForInstance(String servicetype, String id)
+      throws Exception {
     CuratorServiceInstance<Payload> instance = null;
     String path = pathForInstance(servicetype, id);
     try {

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryRestResources.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryRestResources.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryRestResources.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryRestResources.java Fri May 16 16:41:36 2014
@@ -64,7 +64,7 @@ public class RegistryRestResources exten
     this.registry = registry;
   }
 
-  @GET
+//  @GET
   public Response getWadl (@Context HttpServletRequest request) {
     try {
       URI location = new URL(request.getScheme(),
@@ -76,10 +76,23 @@ public class RegistryRestResources exten
       log.error("Error during redirect to WADL", e);
       throw new WebApplicationException(Response.serverError().build());
     }
+  }
 
+  @javax.ws.rs.GET
+  @javax.ws.rs.Produces({MediaType.APPLICATION_JSON})
+  public Response getAtRoot() {
+    try {
+      List<String>
+          instances = registry.serviceTypes();
+      return Response.ok(instances).build();
+    } catch (Exception e) {
+      log.error("Error during generation of response", e);
+      return Response.serverError().build();
+    }
   }
 
-  @Override
+
+    @Override
   @javax.ws.rs.GET
   @javax.ws.rs.Path(SERVICE_NAME)
   @javax.ws.rs.Produces({MediaType.APPLICATION_JSON})
@@ -109,9 +122,10 @@ public class RegistryRestResources exten
       Response.ResponseBuilder builder = Response.ok(instance);
       return builder.build();
     } catch (Exception e) {
-      log.error(String.format("Trying to get instance (%s) from service (%s)",
-                              id,
-                              name), e);
+      log.error("Trying to get instance {} from service {}: {})",
+          id,
+          name,
+          e);
       return Response.serverError().build();
     }
   }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/RegistryViewForProviders.java Fri May 16 16:41:36 2014
@@ -29,4 +29,6 @@ import java.util.List;
 public interface RegistryViewForProviders {
   List<ServiceInstanceData> listInstancesByType(String serviceType) throws
       IOException;
+
+  ServiceInstanceData getSelfRegistration();
 }

Modified: incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java (original)
+++ incubator/slider/trunk/slider-core/src/main/java/org/apache/slider/server/services/registry/SliderRegistryService.java Fri May 16 16:41:36 2014
@@ -20,7 +20,6 @@ package org.apache.slider.server.service
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.x.discovery.ServiceDiscovery;
-import org.apache.curator.x.discovery.ServiceInstance;
 import org.apache.slider.core.registry.info.ServiceInstanceData;
 import org.apache.slider.server.services.curator.CuratorServiceInstance;
 import org.apache.slider.server.services.curator.RegistryBinderService;
@@ -49,17 +48,18 @@ public class SliderRegistryService
 
 
   @Override
-  public List listInstancesByType(String serviceType) throws
+  public List<ServiceInstanceData> listInstancesByType(String serviceType) throws
       IOException {
     List<CuratorServiceInstance<ServiceInstanceData>> services =
         listInstances(serviceType);
-    List payloads = new ArrayList(services.size());
+    List<ServiceInstanceData> payloads = new ArrayList<>(services.size());
     for (CuratorServiceInstance<ServiceInstanceData> instance : services) {
       payloads.add(instance.payload);
     }
     return payloads;
   }
 
+  @Override
   public ServiceInstanceData getSelfRegistration() {
     return selfRegistration;
   }

Modified: incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy (original)
+++ incubator/slider/trunk/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy Fri May 16 16:41:36 2014
@@ -24,7 +24,6 @@ import org.apache.hadoop.service.Lifecyc
 import org.apache.hadoop.service.Service.STATE
 import org.apache.hadoop.service.ServiceStateChangeListener
 import org.apache.hadoop.yarn.api.records.Container
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext
 import org.apache.slider.api.ClusterDescription
 import org.apache.slider.common.tools.SliderFileSystem
 import org.apache.slider.core.conf.AggregateConf
@@ -32,14 +31,16 @@ import org.apache.slider.core.conf.MapOp
 import org.apache.slider.core.exceptions.BadCommandArgumentsException
 import org.apache.slider.core.exceptions.SliderException
 import org.apache.slider.core.launch.ContainerLauncher
-import org.apache.slider.core.registry.info.ServiceInstanceData
 import org.apache.slider.providers.ProviderRole
 import org.apache.slider.providers.ProviderService
 import org.apache.slider.server.appmaster.state.StateAccessForProviders
-import org.apache.slider.server.appmaster.web.rest.agent.*
-import org.apache.slider.server.services.curator.RegistryBinderService
+import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations
+import org.apache.slider.server.appmaster.web.rest.agent.HeartBeat
+import org.apache.slider.server.appmaster.web.rest.agent.HeartBeatResponse
+import org.apache.slider.server.appmaster.web.rest.agent.Register
+import org.apache.slider.server.appmaster.web.rest.agent.RegistrationResponse
+import org.apache.slider.server.appmaster.web.rest.agent.RegistrationStatus
 import org.apache.slider.server.services.registry.RegistryViewForProviders
-import org.apache.slider.server.services.registry.SliderRegistryService
 import org.apache.slider.server.services.utility.EventCallback
 
 class MockProviderService implements ProviderService {
@@ -137,11 +138,6 @@ class MockProviderService implements Pro
     return 0;
   }
 
-  @Override
-  public void buildContainerLaunchContext(ContainerLaunchContext ctx, SliderFileSystem sliderFileSystem, Path generatedConfPath, String role,
-      ClusterDescription clusterSpec, Map<String,String> roleOptions) throws IOException, SliderException {
-  }
-
 
   @Override
   public boolean exec(
@@ -220,4 +216,8 @@ class MockProviderService implements Pro
         }
     }
 
+  @Override
+  void buildEndpointDetails(Map<String, URL> details) {
+
+  }
 }

Modified: incubator/slider/trunk/slider-providers/accumulo/slider-accumulo-provider/src/main/java/org/apache/slider/providers/accumulo/AccumuloProviderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/accumulo/slider-accumulo-provider/src/main/java/org/apache/slider/providers/accumulo/AccumuloProviderService.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/accumulo/slider-accumulo-provider/src/main/java/org/apache/slider/providers/accumulo/AccumuloProviderService.java (original)
+++ incubator/slider/trunk/slider-providers/accumulo/slider-accumulo-provider/src/main/java/org/apache/slider/providers/accumulo/AccumuloProviderService.java Fri May 16 16:41:36 2014
@@ -62,7 +62,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.TreeMap;
 
 
 /**
@@ -124,10 +123,8 @@ public class AccumuloProviderService ext
       Path generatedConfPath,
       MapOperations resourceComponent,
       MapOperations appComponent,
-      Path containerTmpDirPath) throws
-                                                                            IOException,
-      SliderException {
-    
+      Path containerTmpDirPath) throws IOException, SliderException {
+
     this.fileSystem = fileSystem;
     this.instanceDefinition = instanceDefinition;
     
@@ -172,14 +169,19 @@ public class AccumuloProviderService ext
     String heap = "-Xmx" + appComponent.getOption(RoleKeys.JVM_HEAP, DEFAULT_JVM_HEAP);
     String opt = "ACCUMULO_OTHER_OPTS";
     if (SliderUtils.isSet(heap)) {
-      if (AccumuloKeys.ROLE_MASTER.equals(role)) {
-        opt = "ACCUMULO_MASTER_OPTS";
-      } else if (AccumuloKeys.ROLE_TABLET.equals(role)) {
-        opt = "ACCUMULO_TSERVER_OPTS";
-      } else if (AccumuloKeys.ROLE_MONITOR.equals(role)) {
-        opt = "ACCUMULO_MONITOR_OPTS";
-      } else if (AccumuloKeys.ROLE_GARBAGE_COLLECTOR.equals(role)) {
-        opt = "ACCUMULO_GC_OPTS";
+      switch (role) {
+        case AccumuloKeys.ROLE_MASTER:
+          opt = "ACCUMULO_MASTER_OPTS";
+          break;
+        case AccumuloKeys.ROLE_TABLET:
+          opt = "ACCUMULO_TSERVER_OPTS";
+          break;
+        case AccumuloKeys.ROLE_MONITOR:
+          opt = "ACCUMULO_MONITOR_OPTS";
+          break;
+        case AccumuloKeys.ROLE_GARBAGE_COLLECTOR:
+          opt = "ACCUMULO_GC_OPTS";
+          break;
       }
       launcher.setEnv(opt, heap);
     }
@@ -236,7 +238,7 @@ public class AccumuloProviderService ext
 
 
     String accumuloScript = AccumuloClientProvider.buildScriptBinPath(instance);
-    List<String> launchSequence = new ArrayList<String>(8);
+    List<String> launchSequence = new ArrayList<>(8);
     launchSequence.add(0, accumuloScript);
     Collections.addAll(launchSequence, commands);
     return launchSequence;
@@ -290,7 +292,7 @@ public class AccumuloProviderService ext
     } catch (KeeperException e) {
       throw new BadClusterStateException("Failed to connect to Zookeeper at %s after %d seconds",
                                          zkQuorum, timeout);
-    } catch (InterruptedException e) {
+    } catch (InterruptedException ignored) {
       throw new BadClusterStateException(
         "Interrupted while trying to connect to Zookeeper at %s",
         zkQuorum);
@@ -380,7 +382,7 @@ public class AccumuloProviderService ext
   @Override
   public Map<String, String> buildProviderStatus() {
     
-    Map<String,String> status = new HashMap<String, String>();
+    Map<String,String> status = new HashMap<>();
     
     
     return status;
@@ -391,25 +393,29 @@ public class AccumuloProviderService ext
    * @see org.apache.slider.providers.ProviderService#buildMonitorDetails()
    */
   @Override
-  public TreeMap<String,URL> buildMonitorDetails(ClusterDescription clusterDesc) {
-    TreeMap<String,URL> map = new TreeMap<String,URL>();
-    
-    map.put("Active Accumulo Master (RPC): " + getInfoAvoidingNull(clusterDesc, AccumuloKeys.MASTER_ADDRESS), null);
+  public Map<String,URL> buildMonitorDetails(ClusterDescription clusterDesc) {
+    Map<String, URL> details = super.buildMonitorDetails(clusterDesc);
+
+
+    details.put("Active Accumulo Master (RPC): " +
+                   getInfoAvoidingNull(clusterDesc,
+                       AccumuloKeys.MASTER_ADDRESS), null);
     
     String monitorKey = "Active Accumulo Monitor: ";
     String monitorAddr = getInfoAvoidingNull(clusterDesc, AccumuloKeys.MONITOR_ADDRESS);
     if (!StringUtils.isBlank(monitorAddr)) {
       try {
         HostAndPort hostPort = HostAndPort.fromString(monitorAddr);
-        map.put(monitorKey, new URL("http", hostPort.getHostText(), hostPort.getPort(), ""));
+        details.put(monitorKey,
+            new URL("http", hostPort.getHostText(), hostPort.getPort(), ""));
       } catch (Exception e) {
         log.debug("Caught exception parsing Accumulo monitor URL", e);
-        map.put(monitorKey + "N/A", null);
+        details.put(monitorKey + "N/A", null);
       }
     } else {
-      map.put(monitorKey + "N/A", null);
+      details.put(monitorKey + "N/A", null);
     }
 
-    return map;
+    return details;
   }
 }

Added: incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/DemoHBaseCluster.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/DemoHBaseCluster.groovy?rev=1595261&view=auto
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/DemoHBaseCluster.groovy (added)
+++ incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/DemoHBaseCluster.groovy Fri May 16 16:41:36 2014
@@ -0,0 +1,45 @@
+/*
+ * 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.slider.providers.hbase.funtest
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.slider.api.ClusterDescription
+import org.apache.slider.client.SliderClient
+
+class DemoHBaseCluster extends TestFunctionalHBaseCluster {
+
+
+  @Override
+  void clusterOperations(
+      String clustername,
+      SliderClient sliderClient,
+      Configuration clientConf,
+      int numWorkers,
+      Map<String, Integer> roleMap,
+      ClusterDescription cd) {
+    
+  }
+
+  @Override
+  void teardownCluster() {
+    // dont tear it down
+    sleep(5 * 60 * 1000)
+    
+  }
+}

Modified: incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestFunctionalHBaseCluster.groovy
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestFunctionalHBaseCluster.groovy?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestFunctionalHBaseCluster.groovy (original)
+++ incubator/slider/trunk/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/TestFunctionalHBaseCluster.groovy Fri May 16 16:41:36 2014
@@ -60,7 +60,7 @@ public class TestFunctionalHBaseCluster 
   }
 
   public String getClusterZNode() {
-    return "/yarnapps_slider_yarn_" + getClusterName();
+    return "/yarnapps_slider_yarn_" + clusterName;
   }
 
   @Before
@@ -75,9 +75,9 @@ public class TestFunctionalHBaseCluster 
       }
     }, false)
     try {
-      ZKUtil.deleteRecursive(monitor, getClusterZNode())
+      ZKUtil.deleteRecursive(monitor, clusterZNode)
     } catch (KeeperException.NoNodeException ignored) {
-      log.info(getClusterZNode() + " not there")
+      log.info(clusterZNode + " not there")
     }
     setupCluster(clusterName)
   }

Modified: incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java
URL: http://svn.apache.org/viewvc/incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java?rev=1595261&r1=1595260&r2=1595261&view=diff
==============================================================================
--- incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java (original)
+++ incubator/slider/trunk/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java Fri May 16 16:41:36 2014
@@ -57,7 +57,6 @@ import java.net.URL;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.TreeMap;
 
 /**
  * This class implements the server-side aspects
@@ -69,10 +68,8 @@ public class HBaseProviderService extend
     SliderKeys,
     AgentRestOperations{
 
-  public static final String ERROR_UNKNOWN_ROLE = "Unknown role ";
   protected static final Logger log =
     LoggerFactory.getLogger(HBaseProviderService.class);
-  protected static final String NAME = "hbase";
   private static final ProviderUtils providerUtils = new ProviderUtils(log);
   private HBaseClientProvider clientProvider;
   private Configuration siteConf;
@@ -171,16 +168,19 @@ public class HBaseProviderService extend
     String roleCommand;
     String logfile;
     //now look at the role
-    if (ROLE_WORKER.equals(role)) {
-      //role is region server
-      roleCommand = REGION_SERVER;
-      logfile = "/region-server.txt";
-    } else if (ROLE_MASTER.equals(role)) {
-      roleCommand = MASTER;
-      
-      logfile ="/master.txt";
-    } else {
-      throw new SliderInternalStateException("Cannot start role %s", role);
+    switch (role) {
+      case ROLE_WORKER:
+        //role is region server
+        roleCommand = REGION_SERVER;
+        logfile = "/region-server.txt";
+        break;
+      case ROLE_MASTER:
+        roleCommand = MASTER;
+
+        logfile = "/master.txt";
+        break;
+      default:
+        throw new SliderInternalStateException("Cannot start role %s", role);
     }
 
     cli.add(roleCommand);
@@ -260,21 +260,20 @@ public class HBaseProviderService extend
    * @return the provider status - map of entries to add to the info section
    */
   public Map<String, String> buildProviderStatus() {
-    Map<String, String> stats = new HashMap<String, String>();
+    Map<String, String> stats = new HashMap<>();
 
     return stats;
   }
   
-  /* non-javadoc
-   * @see org.apache.slider.providers.ProviderService#buildMonitorDetails()
-   */
+
   @Override
-  public TreeMap<String,URL> buildMonitorDetails(ClusterDescription clusterDesc) {
-    TreeMap<String,URL> map = new TreeMap<String,URL>();
-    
-    map.put("Active HBase Master (RPC): " + getInfoAvoidingNull(clusterDesc, StatusKeys.INFO_MASTER_ADDRESS), null);
+  public Map<String,URL> buildMonitorDetails(ClusterDescription clusterDesc) {
+    Map<String, URL> details = super.buildMonitorDetails(clusterDesc);
+
+    details.put("Active HBase Master (RPC): " 
+                + getInfoAvoidingNull(clusterDesc, StatusKeys.INFO_MASTER_ADDRESS), null);
 
-    return map;
+    return details;
   }
 
   @Override