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 2015/01/09 16:34:25 UTC

[04/10] incubator-slider git commit: SLIDER-719 Create slider REST client library

SLIDER-719 Create slider REST client library


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

Branch: refs/heads/develop
Commit: 372a5b79670059271ad4cab93b04f0ce3512eec2
Parents: 6e1453b
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jan 8 12:33:52 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jan 8 12:33:52 2015 +0000

----------------------------------------------------------------------
 .../apache/slider/core/restclient/HttpVerb.java |  43 ++++++
 .../restclient/UrlConnectionOperations.java     | 149 +++++++++++++++++++
 .../apache/slider/test/SliderTestUtils.groovy   |  51 ++-----
 3 files changed, 206 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/372a5b79/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java b/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
new file mode 100644
index 0000000..6767951
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
@@ -0,0 +1,43 @@
+/*
+ * 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.core.restclient;
+
+public enum HttpVerb {
+  GET("GET", false),
+  POST("POST", true),
+  PUT("POST", true),
+  DELETE("DELETE", false),
+  HEAD("HEAD", false);
+  
+  private final String verb;
+  private final boolean hasUploadBody;
+
+  HttpVerb(String verb, boolean hasUploadBody) {
+    this.verb = verb;
+    this.hasUploadBody = hasUploadBody;
+  }
+
+  public String getVerb() {
+    return verb;
+  }
+
+  public boolean isHasUploadBody() {
+    return hasUploadBody;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/372a5b79/slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java b/slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java
new file mode 100644
index 0000000..eb5d4a7
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java
@@ -0,0 +1,149 @@
+/*
+ * 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.core.restclient;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+/**
+ * Operations on the JDK UrlConnection class. This uses WebHDFS
+ * methods to set up the operations.
+ */
+public class UrlConnectionOperations extends Configured {
+  private static final Logger log =
+      LoggerFactory.getLogger(UrlConnectionOperations.class);
+
+  private URLConnectionFactory connectionFactory;
+
+  public UrlConnectionOperations(Configuration conf) {
+    super(conf);
+    connectionFactory = URLConnectionFactory
+        .newDefaultURLConnectionFactory(conf);  }
+
+  /**
+   * Opens a url with read and connect timeouts
+   *
+   * @param url
+   *          to open
+   * @return URLConnection
+   * @throws IOException
+   */
+  public HttpURLConnection openConnection(URL url, boolean spnego) throws
+      IOException,
+      AuthenticationException {
+    Preconditions.checkArgument(url.getPort() != 0, "no port");
+    HttpURLConnection conn =
+        (HttpURLConnection) connectionFactory.openConnection(url, spnego);
+    conn.setUseCaches(false);
+    conn.setInstanceFollowRedirects(true);
+    return conn;
+  }
+
+  public byte[] execGet(URL url, boolean spnego) throws
+      IOException,
+      AuthenticationException {
+    HttpURLConnection conn = null;
+    int resultCode;
+    byte[] body = null;
+    log.debug("GET {} spnego={}", url, spnego);
+
+    try {
+      conn = openConnection(url, spnego);
+      resultCode = conn.getResponseCode();
+      InputStream stream = conn.getErrorStream();
+      if (stream == null) {
+        stream = conn.getInputStream();
+      }
+      if (stream != null) {
+        // read into a buffer.
+        body = IOUtils.toByteArray(stream);
+      } else {
+        // no body: 
+        log.debug("No body in response");
+
+      }
+    } catch (IOException e) {
+      throw NetUtils.wrapException(url.toString(), 
+          url.getPort(), "localhost", 0, e);
+
+    } catch (AuthenticationException e) {
+      throw new IOException("From " + url + ": " + e.toString(), e);
+
+    } finally {
+      if (conn != null) {
+        conn.disconnect();
+      }
+    }
+    uprateFaults(url.toString(), resultCode, body);
+    return body;
+  }
+
+  /**
+   * Uprate error codes 400 and up into faults; 
+   * 404 is converted to a {@link NotFoundException},
+   * 401 to {@link ForbiddenException}
+   * @param url URL as string
+   * @param resultCode response from the request
+   * @param body optional body of the request
+   * @throws IOException if the result was considered a failure
+   */
+  public static void uprateFaults(String url,
+      int resultCode, byte[] body)
+      throws IOException {
+
+    if (resultCode < 400) {
+      //success
+      return;
+    }
+    if (resultCode == 404) {
+      throw new NotFoundException(url);
+    }
+    if (resultCode == 401) {
+      throw new ForbiddenException(url);
+    }
+    // all other error codes
+    String bodyAsString;
+    if (body != null && body.length > 0) {
+      bodyAsString = new String(body);
+    } else {
+      bodyAsString = "";
+    }
+    String message = "Request to " + url +
+                     " failed with exit code " + resultCode
+                     + ", body length " + bodyAsString.length()
+                     + ":\n" + bodyAsString;
+    log.error(message);
+    throw new IOException(message);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/372a5b79/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
index cd78c6c..b496d45 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/SliderTestUtils.groovy
@@ -56,6 +56,7 @@ import org.apache.slider.core.main.ServiceLaunchException
 import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.core.persist.JsonSerDeser
 import org.apache.slider.core.registry.docstore.PublishedConfigSet
+import org.apache.slider.core.restclient.UrlConnectionOperations
 import org.apache.slider.server.appmaster.web.HttpCacheHeaders
 import org.apache.slider.server.appmaster.web.rest.RestPaths
 import org.apache.slider.server.services.workflow.ForkedProcessService
@@ -500,7 +501,7 @@ class SliderTestUtils extends Assert {
     try {
       resultCode = client.executeMethod(get);
     } catch (IOException e) {
-      throw NetUtils.wrapException(url, 0, null, 0, e)
+      throw NetUtils.wrapException(url, destURI.port, "localhost", 0, e)
     }
 
     def body = get.responseBodyAsString
@@ -518,15 +519,18 @@ class SliderTestUtils extends Assert {
   public static void uprateFaults(String url, int resultCode, String body) {
 
     if (resultCode == 404) {
-      throw new NotFoundException(url)
+      throw new NotFoundException(url);
     }
     if (resultCode == 401) {
-      throw new ForbiddenException(url)
+      throw new ForbiddenException(url);
     }
     if (!(resultCode >= 200 && resultCode < 400)) {
-      def message = "Request to $url failed with exit code $resultCode, body length ${body?.length()}:\n$body"
-      log.error(message)
-      throw new IOException(message)
+      String message = "Request to " + url +
+                       " failed with exit code " +
+                       resultCode + ", body length " +
+                       body?.length() + ":\n" + "body;"
+      log.error(message);
+      throw new IOException(message);
     }
   }
 
@@ -566,11 +570,10 @@ class SliderTestUtils extends Assert {
     throw ex;
   } 
 
-  static URLConnectionFactory connectionFactory
+  static UrlConnectionOperations connectionFactory
 
   public static def initConnectionFactory(Configuration conf) {
-    connectionFactory = URLConnectionFactory
-        .newDefaultURLConnectionFactory(conf);
+    connectionFactory = new UrlConnectionOperations(conf);
   }
 
 
@@ -593,34 +596,8 @@ class SliderTestUtils extends Assert {
 
     log.info("Fetching HTTP content at " + path);
     URL url = new URL(path)
-    assert url.port != 0
-    HttpURLConnection conn = null;
-    int resultCode = 0
-    def body = ""
-    try {
-      conn = (HttpURLConnection) connectionFactory.openConnection(url);
-      conn.instanceFollowRedirects = true;
-      conn.connect()
-      
-
-      resultCode = conn.responseCode
-      
-      if (connectionChecks) {
-        connectionChecks(conn)
-      }
-      
-      InputStream stream = conn.errorStream;
-      if (stream == null) {
-        stream = conn.inputStream;
-      }
-
-      body = stream ? stream.text : "(no body)"
-    } catch (IOException e) {
-      throw NetUtils.wrapException(url.toString(), 0, null, 0, e)
-    } finally {
-      conn?.disconnect()
-    }
-    uprateFaults(path, resultCode, body)
+    def bytes = connectionFactory.execGet(url, false)
+    String body = new String(bytes)
     return body;
   }