You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@shindig.apache.org by lr...@apache.org on 2008/08/20 06:14:41 UTC

svn commit: r687235 [2/2] - in /incubator/shindig/trunk: config/ features/ features/opensocial-current/ java/gadgets/src/main/java/org/apache/shindig/gadgets/servlet/ java/server/src/main/webapp/WEB-INF/ java/server/src/test/java/org/apache/shindig/ser...

Modified: incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RequestItem.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RequestItem.java?rev=687235&r1=687234&r2=687235&view=diff
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RequestItem.java (original)
+++ incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RequestItem.java Tue Aug 19 21:14:39 2008
@@ -22,29 +22,18 @@
 import org.apache.shindig.social.opensocial.spi.PersonService;
 import org.apache.shindig.social.opensocial.spi.UserId;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import com.google.common.collect.Lists;
 
-import org.apache.commons.io.IOUtils;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Enumeration;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
-import javax.servlet.ServletInputStream;
-import javax.servlet.http.HttpServletRequest;
-
 /**
- * Represents the request items that come from the restful request.
+ * Abstract base type for social API requests.
  */
-public class RequestItem {
+public abstract class RequestItem {
 
-  // Common OpenSocial RESTful fields
+  // Common OpenSocial API fields
   public static final String APP_ID = "appId";
 
   public static final String USER_ID = "userId";
@@ -61,129 +50,30 @@
 
   public static final String FIELDS = "fields";
 
-  // OpenSocial defaults
+  // Opensocial defaults
   public static final int DEFAULT_START_INDEX = 0;
 
   public static final int DEFAULT_COUNT = 20;
 
   public static final String APP_SUBSTITUTION_TOKEN = "@app";
 
-  private String url;
+  private final SecurityToken token;
 
-  private String method;
+  protected final BeanConverter converter;
 
-  private Map<String, List<String>> params;
+  private final String operation;
 
-  private String postData;
-
-  private SecurityToken token;
-
-  private BeanConverter converter;
-
-  public RequestItem() {
-    params = Maps.newHashMap();
-  }
+  private final String service;
 
-  public RequestItem(HttpServletRequest servletRequest, SecurityToken token, String method,
-      BeanConverter converter) {
-    this.url = servletRequest.getPathInfo();
-    this.params = createParameterMap(servletRequest);
+  public RequestItem(String service, String operation, SecurityToken token, BeanConverter converter) {
+    this.service = service;
+    this.operation = operation;
     this.token = token;
-
-    this.method = method;
     this.converter = converter;
-
-    try {
-      ServletInputStream is = servletRequest.getInputStream();
-      postData = new String(IOUtils.toByteArray(is));
-    } catch (IOException e) {
-      throw new RuntimeException("Could not get the post data from the request", e);
-    }
-  }
-
-  private static Map<String, List<String>> createParameterMap(HttpServletRequest servletRequest) {
-    Map<String, List<String>> parameters = Maps.newHashMap();
-
-    Enumeration names = servletRequest.getParameterNames();
-    while (names.hasMoreElements()) {
-      String name = (String) names.nextElement();
-      String[] paramValues = servletRequest.getParameterValues(name);
-      parameters.put(name, Lists.newArrayList(paramValues));
-    }
-
-    return parameters;
-  }
-
-  /*
-   * Takes any url params out of the url and puts them into the param map.
-   * Usually the servlet request code does this for us but the batch request calls have to do it
-   * by hand.
-   */
-  void putUrlParamsIntoParameters() {
-    if (this.params == null) {
-      this.params = Maps.newHashMap();
-    }
-
-    String fullUrl = this.url;
-    int queryParamIndex = fullUrl.indexOf('?');
-
-    if (queryParamIndex != -1) {
-      this.url = fullUrl.substring(0, queryParamIndex);
-
-      String queryParams = fullUrl.substring(queryParamIndex + 1);
-      for (String param : queryParams.split("&")) {
-        String[] paramPieces = param.split("=", 2);
-        List<String> paramList = this.params.get(paramPieces[0]);
-        if (paramList == null) {
-          paramList = Lists.newArrayListWithCapacity(1);
-          this.params.put(paramPieces[0], paramList);
-        }
-        if (paramPieces.length == 2) {
-          paramList.add(paramPieces[1]);
-        } else {
-          paramList.add("");
-        }
-      }
-    }
-  }
-
-  /**
-   * This could definitely be cleaner.. TODO: Come up with a cleaner way to handle all of this
-   * code.
-   *
-   * @param urlTemplate The template the url follows
-   */
-  public void parseUrlWithTemplate(String urlTemplate) {
-    this.putUrlParamsIntoParameters();
-
-    String[] actualUrl = this.url.split("/");
-    String[] expectedUrl = urlTemplate.split("/");
-
-    for (int i = 0; i < actualUrl.length; i++) {
-      String actualPart = actualUrl[i];
-      String expectedPart = expectedUrl[i];
-
-      if (expectedPart.startsWith("{")) {
-        if (expectedPart.endsWith("}+")) {
-          // The param can be a repeated field. Use ',' as default separator
-          this.params
-              .put(expectedPart.substring(1, expectedPart.length() - 2),
-                  Lists.newArrayList(actualPart.split(",")));
-        } else {
-          if (actualPart.indexOf(',') != -1) {
-            throw new IllegalArgumentException("Cannot expect plural value " + actualPart
-                + " for singular field " + expectedPart + " in " + expectedUrl);
-          }
-          this.params.put(expectedPart.substring(1, expectedPart.length() - 1),
-              Lists.newArrayList(actualPart));
-        }
-      }
-    }
   }
 
   public String getAppId() {
     String appId = getParameter(APP_ID);
-
     if (appId != null && appId.equals(APP_SUBSTITUTION_TOKEN)) {
       return token.getAppId();
     } else {
@@ -193,20 +83,30 @@
 
   public Set<UserId> getUsers() {
     List<String> ids = getListParameter(USER_ID);
-    Set<UserId> returnVal = Sets.newLinkedHashSet();
+    if (ids.isEmpty()) {
+      if (token.getViewerId() != null) {
+        // Assume @me
+        ids = Lists.newArrayList(token.getViewerId());
+      } else {
+        throw new IllegalArgumentException("No userId provided and viewer not available");
+      }
+    }
+    Set<UserId> userIds = Sets.newLinkedHashSet();
     for (String id : ids) {
-      returnVal.add(UserId.fromJson(id));
+      userIds.add(UserId.fromJson(id));
     }
-    return returnVal;
+    return userIds;
   }
 
+
   public GroupId getGroup() {
-    return GroupId.fromJson(getParameter(GROUP_ID));
+    return GroupId.fromJson(getParameter(GROUP_ID, "@self"));
   }
 
   public int getStartIndex() {
     String startIndex = getParameter(START_INDEX);
-    return startIndex == null ? DEFAULT_START_INDEX : Integer.valueOf(startIndex);
+    return startIndex == null ? DEFAULT_START_INDEX
+        : Integer.valueOf(startIndex);
   }
 
   public int getCount() {
@@ -229,100 +129,37 @@
   }
 
   public Set<String> getFields() {
-    return getFields(Collections.<String>emptySet());
+    return getFields(Sets.<String>newHashSet());
   }
 
   public Set<String> getFields(Set<String> defaultValue) {
-    List<String> paramValue = getListParameter(FIELDS);
-    if (!paramValue.isEmpty()) {
-      return Sets.newHashSet(paramValue);
+    Set result = Sets.newHashSet();
+    result.addAll(getListParameter(FIELDS));
+    if (result.isEmpty()) {
+      return defaultValue;
     }
-    return defaultValue;
+    return result;
   }
 
-  public <T> T getPostData(Class<T> postDataClass) {
-    return converter.convertToObject(postData, postDataClass);
+  public String getOperation() {
+    return operation;
   }
 
-  public String getUrl() {
-    return url;
-  }
-
-  public void setUrl(String url) {
-    this.url = url;
-  }
-
-  public String getMethod() {
-    return method;
-  }
-
-  public void setMethod(String method) {
-    this.method = method;
-  }
-
-  Map<String, List<String>> getParameters() {
-    return params;
-  }
-
-  void setParameter(String paramName, String paramValue) {
-    // Ignore nulls
-    if (paramValue == null) {
-      return;
-    }
-    this.params.put(paramName, Lists.newArrayList(paramValue));
-  }
-
-  void setListParameter(String paramName, List<String> paramValue) {
-    this.params.put(paramName, paramValue);
-  }
-
-  /**
-   * Return a single param value
-   */
-  public String getParameter(String paramName) {
-    List<String> paramValue = this.params.get(paramName);
-    if (paramValue != null && !paramValue.isEmpty()) {
-      return paramValue.get(0);
-    }
-    return null;
-  }
-
-  /**
-   * Return a list param value
-   */
-  public List<String> getListParameter(String paramName) {
-    List<String> stringList = this.params.get(paramName);
-    if (stringList == null) {
-      return Collections.emptyList();
-    }
-    if (stringList.size() == 1 && stringList.get(0).indexOf(',') != -1) {
-      stringList = Arrays.asList(stringList.get(0).split(","));
-      this.params.put(paramName, stringList);
-    }
-    return stringList;
-  }
-
-  public String getPostData() {
-    return postData;
-  }
-
-  public void setPostData(String postData) {
-    this.postData = postData;
+  public String getService() {
+    return service;
   }
 
   public SecurityToken getToken() {
     return token;
   }
 
-  public void setToken(SecurityToken token) {
-    this.token = token;
-  }
+  public abstract <T> T getTypedParameter(String parameterName, Class<T> postDataClass);
 
-  public BeanConverter getConverter() {
-    return converter;
-  }
+  public abstract void applyUrlTemplate(String urlTemplate);
 
-  public void setConverter(BeanConverter converter) {
-    this.converter = converter;
-  }
+  public abstract String getParameter(String paramName);
+
+  public abstract String getParameter(String paramName, String defaultValue);
+
+  public abstract List<String> getListParameter(String paramName);
 }

Added: incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RestfulRequestItem.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RestfulRequestItem.java?rev=687235&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RestfulRequestItem.java (added)
+++ incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RestfulRequestItem.java Tue Aug 19 21:14:39 2008
@@ -0,0 +1,234 @@
+/*
+ * 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.shindig.social.opensocial.service;
+
+import org.apache.shindig.common.SecurityToken;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.Map;
+
+import javax.servlet.ServletInputStream;
+import javax.servlet.http.HttpServletRequest;
+
+/**
+ * Represents the request items that come from the restful request.
+ */
+public class RestfulRequestItem extends RequestItem {
+
+  protected static final String X_HTTP_METHOD_OVERRIDE = "X-HTTP-Method-Override";
+
+  private String url;
+
+  private Map<String, List<String>> params;
+
+  private String postData;
+
+  public RestfulRequestItem(String service, String method,
+      SecurityToken token, BeanConverter converter) {
+    super(service, method, token, converter);
+  }
+
+  public RestfulRequestItem(String path, String method, String postData, SecurityToken token,
+      BeanConverter converter) {
+    super(getServiceFromPath(path), method, token, converter);
+    this.postData = postData;
+    this.url = path;
+    putUrlParamsIntoParameters();
+  }
+
+  public RestfulRequestItem(HttpServletRequest servletRequest, SecurityToken token,
+      BeanConverter converter) {
+    super(getServiceFromPath(servletRequest.getPathInfo()),
+        getMethod(servletRequest),
+        token, converter);
+    this.url = servletRequest.getPathInfo();
+    this.params = createParameterMap(servletRequest);
+
+    try {
+      ServletInputStream is = servletRequest.getInputStream();
+      postData = new String(IOUtils.toByteArray(is));
+    } catch (IOException e) {
+      throw new RuntimeException("Could not get the post data from the request", e);
+    }
+  }
+
+  static String getServiceFromPath(String pathInfo) {
+    pathInfo = pathInfo.substring(1);
+    int indexOfNextPathSeparator = pathInfo.indexOf('/');
+    if (indexOfNextPathSeparator != -1) {
+      return pathInfo.substring(0, indexOfNextPathSeparator);
+    }
+    return pathInfo;
+  }
+
+  static String getMethod(HttpServletRequest request) {
+    String override = request.getParameter(X_HTTP_METHOD_OVERRIDE);
+    if (!StringUtils.isBlank(override)) {
+      return override;
+    } else {
+      return request.getMethod();
+    }
+  }
+
+  private static Map<String, List<String>> createParameterMap(HttpServletRequest servletRequest) {
+    Map<String, List<String>> parameters = Maps.newHashMap();
+
+    Enumeration names = servletRequest.getParameterNames();
+    while (names.hasMoreElements()) {
+      String name = (String) names.nextElement();
+      String[] paramValues = servletRequest.getParameterValues(name);
+      parameters.put(name, Lists.newArrayList(paramValues));
+    }
+    return parameters;
+  }
+
+  /*
+   * Takes any url params out of the url and puts them into the param map.
+   * Usually the servlet request code does this for us but the batch request calls have to do it
+   * by hand.
+   */
+  void putUrlParamsIntoParameters() {
+    if (this.params == null) {
+      this.params = Maps.newHashMap();
+    }
+
+    String fullUrl = this.url;
+    int queryParamIndex = fullUrl.indexOf('?');
+
+    if (queryParamIndex != -1) {
+      this.url = fullUrl.substring(0, queryParamIndex);
+
+      String queryParams = fullUrl.substring(queryParamIndex + 1);
+      for (String param : queryParams.split("&")) {
+        String[] paramPieces = param.split("=", 2);
+        List<String> paramList = this.params.get(paramPieces[0]);
+        if (paramList == null) {
+          paramList = Lists.newArrayListWithCapacity(1);
+          this.params.put(paramPieces[0], paramList);
+        }
+        if (paramPieces.length == 2) {
+          paramList.add(paramPieces[1]);
+        } else {
+          paramList.add("");
+        }
+      }
+    }
+  }
+
+  /**
+   * This could definitely be cleaner.. TODO: Come up with a cleaner way to handle all of this
+   * code.
+   *
+   * @param urlTemplate The template the url follows
+   */
+  public void applyUrlTemplate(String urlTemplate) {
+    this.putUrlParamsIntoParameters();
+
+    String[] actualUrl = this.url.split("/");
+    String[] expectedUrl = urlTemplate.split("/");
+
+    for (int i = 0; i < actualUrl.length; i++) {
+      String actualPart = actualUrl[i];
+      String expectedPart = expectedUrl[i];
+
+      if (expectedPart.startsWith("{")) {
+        if (expectedPart.endsWith("}+")) {
+          // The param can be a repeated field. Use ',' as default separator
+          this.params
+              .put(expectedPart.substring(1, expectedPart.length() - 2),
+                  Lists.newArrayList(actualPart.split(",")));
+        } else {
+          if (actualPart.indexOf(',') != -1) {
+            throw new IllegalArgumentException("Cannot expect plural value " + actualPart
+                + " for singular field " + expectedPart + " in " + this.url);
+          }
+          this.params.put(expectedPart.substring(1, expectedPart.length() - 1),
+              Lists.newArrayList(actualPart));
+        }
+      }
+    }
+  }
+
+
+  public <T> T getTypedParameter(String parameterName, Class<T> postDataClass) {
+    // We assume the the only typed parameter in a restful request is the post-content
+    // and so we simply ignore the parameter name
+    return converter.convertToObject(postData, postDataClass);
+  }
+
+
+  Map<String, List<String>> getParameters() {
+    return params;
+  }
+
+  void setParameter(String paramName, String paramValue) {
+    // Ignore nulls
+    if (paramValue == null) {
+      return;
+    }
+    this.params.put(paramName, Lists.newArrayList(paramValue));
+  }
+
+  void setListParameter(String paramName, List<String> paramValue) {
+    this.params.put(paramName, paramValue);
+  }
+
+  /**
+   * Return a single param value
+   */
+  public String getParameter(String paramName) {
+    List<String> paramValue = this.params.get(paramName);
+    if (paramValue != null && !paramValue.isEmpty()) {
+      return paramValue.get(0);
+    }
+    return null;
+  }
+
+  public String getParameter(String paramName, String defaultValue) {
+    String result = getParameter(paramName);
+    if (result == null) {
+      return defaultValue;
+    }
+    return result;
+  }
+
+  /**
+   * Return a list param value
+   */
+  public List<String> getListParameter(String paramName) {
+    List<String> stringList = this.params.get(paramName);
+    if (stringList == null) {
+      return Collections.emptyList();
+    }
+    if (stringList.size() == 1 && stringList.get(0).indexOf(',') != -1) {
+      stringList = Arrays.asList(stringList.get(0).split(","));
+      this.params.put(paramName, stringList);
+    }
+    return stringList;
+  }
+}

Added: incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RpcRequestItem.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RpcRequestItem.java?rev=687235&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RpcRequestItem.java (added)
+++ incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/opensocial/service/RpcRequestItem.java Tue Aug 19 21:14:39 2008
@@ -0,0 +1,134 @@
+/*
+ * 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.shindig.social.opensocial.service;
+
+import org.apache.shindig.common.SecurityToken;
+
+import com.google.common.collect.Lists;
+
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A JSON-RPC specific implementation of RequestItem
+ */
+public class RpcRequestItem extends RequestItem {
+
+  private JSONObject data;
+
+  static String getService(String rpcMethod) {
+    return rpcMethod.substring(0, rpcMethod.indexOf('.'));
+  }
+
+  static String getOperation(String rpcMethod) {
+    return rpcMethod.substring(rpcMethod.indexOf('.') + 1);
+  }
+
+  public RpcRequestItem(JSONObject rpc, SecurityToken token,
+      BeanConverter converter) throws JSONException {
+    super(getService(rpc.getString("method")),
+        getOperation(rpc.getString("method")),
+        token, converter);
+    if (rpc.has("params")) {
+      this.data = rpc.getJSONObject("params");
+    } else {
+      this.data = new JSONObject();
+    }
+  }
+
+  public String getParameter(String paramName) {
+    try {
+      if (data.has(paramName)) {
+        return data.getString(paramName);
+      } else {
+        return null;
+      }
+    } catch (JSONException je) {
+      throw new IllegalArgumentException(je);
+    }
+  }
+
+  public String getParameter(String paramName, String defaultValue) {
+    try {
+      if (data.has(paramName)) {
+        return data.getString(paramName);
+      } else {
+        return defaultValue;
+      }
+    } catch (JSONException je) {
+      throw new IllegalArgumentException(je);
+    }
+  }
+
+  public List<String> getListParameter(String paramName) {
+    try {
+      if (data.has(paramName)) {
+        if (data.get(paramName) instanceof JSONArray) {
+          JSONArray jsonArray = data.getJSONArray(paramName);
+          List<String> returnVal = Lists.newArrayListWithExpectedSize(jsonArray.length());
+          for (int i = 0; i < jsonArray.length(); i++) {
+            returnVal.add(jsonArray.getString(i));
+          }
+          return returnVal;
+        } else {
+          // Allow up-conversion of non-array to array params.
+          return Lists.newArrayList(data.getString(paramName));
+        }
+      } else {
+        return Collections.emptyList();
+      }
+    } catch (JSONException je) {
+      throw new IllegalArgumentException(je);
+    }
+  }
+
+  public <T> T getTypedParameter(String parameterName, Class<T> dataTypeClass) {
+    try {
+      return converter.convertToObject(data.get(parameterName).toString(), dataTypeClass);
+    } catch (JSONException je) {
+      throw new IllegalArgumentException(je);
+    }
+  }
+
+
+  public void applyUrlTemplate(String urlTemplate) {
+    // No params in the URL
+  }
+
+  void setParameter(String paramName, String param) {
+    try {
+      data.put(paramName, param);
+    } catch (JSONException je) {
+      throw new IllegalArgumentException(je);
+    }
+  }
+
+  void setListParameter(String paramName, List<String> params) {
+    try {
+      JSONArray arr = new JSONArray(params);
+      data.put(paramName, arr);
+    } catch (JSONException je) {
+      throw new IllegalArgumentException(je);
+    }
+  }
+}

Modified: incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/sample/service/SampleContainerHandler.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/sample/service/SampleContainerHandler.java?rev=687235&r1=687234&r2=687235&view=diff
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/sample/service/SampleContainerHandler.java (original)
+++ incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/sample/service/SampleContainerHandler.java Tue Aug 19 21:14:39 2008
@@ -68,7 +68,7 @@
   protected Future<? extends ResponseItem> handlePost(RequestItem request) {
     ResponseItem<Object> response = new ResponseItem<Object>("");
 
-    request.parseUrlWithTemplate(POST_PATH);
+    request.applyUrlTemplate(POST_PATH);
     String type = request.getParameter("type");
     if (type.equals("setstate")) {
       try {

Modified: incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/ActivityHandlerTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/ActivityHandlerTest.java?rev=687235&r1=687234&r2=687235&view=diff
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/ActivityHandlerTest.java (original)
+++ incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/ActivityHandlerTest.java Tue Aug 19 21:14:39 2008
@@ -28,14 +28,12 @@
 import org.apache.shindig.social.opensocial.spi.RestfulCollection;
 import org.apache.shindig.social.opensocial.spi.UserId;
 
-import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 import org.easymock.classextension.EasyMock;
 
 import junit.framework.TestCase;
 
-import java.util.Map;
 import java.util.Set;
 
 public class ActivityHandlerTest extends TestCase {
@@ -48,7 +46,7 @@
 
   private FakeGadgetToken token;
 
-  private RequestItem request;
+  private RestfulRequestItem request;
 
   private static final Set<UserId> JOHN_DOE = Sets
       .newHashSet(new UserId(UserId.Type.userId, "john.doe"));
@@ -77,16 +75,7 @@
   }
 
   private void setPathAndPostData(String path, String postData) {
-    Map<String, String> params = Maps.newHashMap();
-
-    request = new RequestItem();
-    request.setUrl(path);
-    for (Map.Entry<String, String> entry : params.entrySet()) {
-      request.setParameter(entry.getKey(), entry.getValue());
-    }
-    request.setToken(token);
-    request.setConverter(converter);
-    request.setPostData(postData);
+    request = new RestfulRequestItem(path, "GET", postData, token, converter);
   }
 
   private void assertHandleGetForGroup(GroupId.Type group) throws Exception {
@@ -185,4 +174,4 @@
     assertEquals(data, handler.handleDelete(request).get());
     verify();
   }
-}
\ No newline at end of file
+}

Modified: incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/AppDataHandlerTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/AppDataHandlerTest.java?rev=687235&r1=687234&r2=687235&view=diff
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/AppDataHandlerTest.java (original)
+++ incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/AppDataHandlerTest.java Tue Aug 19 21:14:39 2008
@@ -48,7 +48,8 @@
 
   private FakeGadgetToken token;
 
-  private RequestItem request;
+  private RestfulRequestItem request;
+
 
   private static final Set<UserId> JOHN_DOE = Collections.unmodifiableSet(Sets
       .newHashSet(new UserId(UserId.Type.userId, "john.doe")));
@@ -84,14 +85,10 @@
   }
 
   private void setPathAndParams(String path, Map<String, String> params, String postData) {
-    request = new RequestItem();
-    request.setUrl(path);
+    request = new RestfulRequestItem(path, "GET", postData, token, converter);
     for (Map.Entry<String, String> entry : params.entrySet()) {
       request.setParameter(entry.getKey(), entry.getValue());
     }
-    request.setToken(token);
-    request.setConverter(converter);
-    request.setPostData(postData);
   }
 
   private void assertHandleGetForGroup(GroupId.Type group) throws Exception {
@@ -195,4 +192,4 @@
     assertEquals(data, handler.handleDelete(request).get());
     verify();
   }
-}
\ No newline at end of file
+}

Modified: incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/DataServiceServletTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/DataServiceServletTest.java?rev=687235&r1=687234&r2=687235&view=diff
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/DataServiceServletTest.java (original)
+++ incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/DataServiceServletTest.java Tue Aug 19 21:14:39 2008
@@ -29,11 +29,11 @@
 
 import com.google.inject.Guice;
 import com.google.inject.Injector;
-import junit.framework.TestCase;
+
 import org.easymock.classextension.EasyMock;
 
-import javax.servlet.ServletInputStream;
-import javax.servlet.http.HttpServletResponse;
+import junit.framework.TestCase;
+
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.StringTokenizer;
@@ -42,7 +42,11 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import javax.servlet.ServletInputStream;
+import javax.servlet.http.HttpServletResponse;
+
 public class DataServiceServletTest extends TestCase {
+
   private static final FakeGadgetToken FAKE_GADGET_TOKEN = new FakeGadgetToken()
       .setOwnerId("john.doe").setViewerId("john.doe");
 
@@ -114,14 +118,16 @@
     verifyHandlerWasFoundForPathInfo(route, appDataHandler, "PUT", null, "PUT");
   }
 
-  /** Tests a data handler that returns a failed Future */
+  /**
+   * Tests a data handler that returns a failed Future
+   */
   public void testFailedRequest() throws Exception {
     String route = '/' + DataServiceServlet.APPDATA_ROUTE;
     setupRequest(route, "GET", null);
     EasyMock.expect(injector.getInstance(AppDataHandler.class)).andStubReturn(appDataHandler);
     setupInjector();
 
-    EasyMock.expect(appDataHandler.handleItem(EasyMock.isA(RequestItem.class)));
+    EasyMock.expect(appDataHandler.handleItem(EasyMock.isA(RestfulRequestItem.class)));
     EasyMock.expectLastCall().andReturn(new FailingFuture());
 
     res.sendError(500, "FAILED");
@@ -169,7 +175,7 @@
     EasyMock.expect(req.getPathInfo()).andStubReturn(pathInfo);
     EasyMock.expect(req.getMethod()).andStubReturn(actualMethod);
     EasyMock.expect(req.getParameterNames()).andStubReturn(new StringTokenizer(""));
-    EasyMock.expect(req.getParameter(DataServiceServlet.X_HTTP_METHOD_OVERRIDE)).andReturn(
+    EasyMock.expect(req.getParameter(RestfulRequestItem.X_HTTP_METHOD_OVERRIDE)).andReturn(
         overrideMethod);
     EasyMock.expect(req.getParameter(DataServiceServlet.FORMAT_PARAM)).andReturn(null);
 
@@ -177,40 +183,10 @@
   }
 
   public void testInvalidRoute() throws Exception {
-    RequestItem requestItem = new RequestItem();
-    requestItem.setUrl("/ahhh!");
-
+    RestfulRequestItem requestItem = new RestfulRequestItem("/ahhh!", "GET", null,
+        FAKE_GADGET_TOKEN, jsonConverter);
     ResponseItem responseItem = servlet.handleRequestItem(requestItem).get();
-    assertEquals(ResponseError.BAD_REQUEST, responseItem.getError());
-  }
-
-  public void testGetHttpMethodFromParameter() throws Exception {
-    String method = "POST";
-    assertEquals(method, servlet.getHttpMethodFromParameter(method, null));
-    assertEquals(method, servlet.getHttpMethodFromParameter(method, ""));
-    assertEquals(method, servlet.getHttpMethodFromParameter(method, "  "));
-    assertEquals("DELETE", servlet.getHttpMethodFromParameter(method, "DELETE"));
-  }
-
-  public void testRouteFromParameter() throws Exception {
-    assertEquals("path", servlet.getRouteFromParameter("/path"));
-    assertEquals("path", servlet.getRouteFromParameter("/path/fun"));
-    assertEquals("path", servlet.getRouteFromParameter("/path/fun/yes"));
-  }
-
-  public void testIsBatchUrl() throws Exception {
-    assertBatchUrl("/jsonBatch", true);
-    assertBatchUrl("/path/to/the/jsonBatch", true);
-    assertBatchUrl("/people/normalpath", false);
-    assertBatchUrl("/notjsonBatch", false);
-  }
-
-  private void assertBatchUrl(String url, boolean isBatch) {
-    EasyMock.expect(req.getPathInfo()).andReturn(url);
-    EasyMock.replay(req);
-    assertEquals(isBatch, servlet.isBatchUrl(req));
-    EasyMock.verify(req);
-    EasyMock.reset(req);
+    assertEquals(ResponseError.NOT_IMPLEMENTED, responseItem.getError());
   }
 
   public void testGetConverterForRequest() throws Exception {
@@ -237,6 +213,7 @@
    * Future implementation that fails with an exception.
    */
   private static class FailingFuture implements Future<ResponseItem> {
+
     public boolean cancel(boolean mayInterruptIfRunning) {
       return false;
     }

Added: incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/JsonConversionUtilsTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/JsonConversionUtilsTest.java?rev=687235&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/JsonConversionUtilsTest.java (added)
+++ incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/JsonConversionUtilsTest.java Tue Aug 19 21:14:39 2008
@@ -0,0 +1,142 @@
+/*
+ * 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.shindig.social.opensocial.service;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import junit.framework.TestCase;
+
+import java.util.Iterator;
+
+/**
+ * Test for conversion of a structured key-value set to a JSON object
+ */
+public class JsonConversionUtilsTest extends TestCase {
+
+  public JsonConversionUtilsTest() {
+  }
+
+  public void testSimplePathToJsonParsing()
+      throws Exception {
+    JSONObject root = new JSONObject();
+    JsonConversionUtils.buildHolder(root, "a.a.a".split("\\."), 0);
+    assertJsonEquals(root, new JSONObject("{a:{a:{}}}"));
+  }
+
+  public void testArrayPathToJsonParsing()
+      throws Exception {
+    JSONObject root = new JSONObject();
+    JsonConversionUtils.buildHolder(root, "a.a(0).a".split("\\."), 0);
+    JsonConversionUtils.buildHolder(root, "a.a(1).a".split("\\."), 0);
+    JsonConversionUtils.buildHolder(root, "a.a(2).a".split("\\."), 0);
+    assertJsonEquals(root, new JSONObject("{a:{a:[{},{},{}]}}"));
+  }
+
+  public void testValueToJsonParsing()
+      throws Exception {
+    assertJsonEquals(JsonConversionUtils.convertToJsonValue("abc"), "abc");
+    assertJsonEquals(JsonConversionUtils.convertToJsonValue("\"a,b,c\""), "a,b,c");
+    assertJsonEquals(JsonConversionUtils.convertToJsonValue("true"), true);
+    assertJsonEquals(JsonConversionUtils.convertToJsonValue("false"), false);
+    assertJsonEquals(JsonConversionUtils.convertToJsonValue("null"), JSONObject.NULL);
+    assertJsonEquals(JsonConversionUtils.convertToJsonValue("'abc'"), "abc");
+    assertJsonEquals(JsonConversionUtils.convertToJsonValue("a,b,c"),
+        new JSONArray(Lists.newArrayList("a", "b", "c")));
+    assertJsonEquals(JsonConversionUtils.convertToJsonValue("1,2,3,true,false,null"),
+        new JSONArray(Lists.newArrayList(1, 2, 3, true,
+            false, null)));
+    assertJsonEquals(JsonConversionUtils.convertToJsonValue("(1)"),
+        new JSONArray(Lists.newArrayList(1)));
+    assertJsonEquals(JsonConversionUtils.convertToJsonValue("(true)"),
+        new JSONArray(Lists.newArrayList(true)));
+  }
+
+  public void testParameterMapToJsonParsing()
+      throws Exception {
+    assertJsonEquals(JsonConversionUtils.parametersToJsonObject(Maps.immutableMap("a.b.c", "1")),
+        new JSONObject("{a:{b:{c:1}}}"));
+    assertJsonEquals(
+        JsonConversionUtils.parametersToJsonObject(Maps.immutableMap("a.b.c", "\"1\"")),
+        new JSONObject("{a:{b:{c:\"1\"}}}"));
+    assertJsonEquals(JsonConversionUtils.parametersToJsonObject(Maps.immutableMap("a.b.c", "true")),
+        new JSONObject("{a:{b:{c:true}}}"));
+    assertJsonEquals(
+        JsonConversionUtils.parametersToJsonObject(Maps.immutableMap("a.b.c", "false")),
+        new JSONObject("{a:{b:{c:false}}}"));
+    assertJsonEquals(JsonConversionUtils.parametersToJsonObject(Maps.immutableMap("a.b.c", "null")),
+        new JSONObject("{a:{b:{c:null}}}"));
+    assertJsonEquals(JsonConversionUtils.parametersToJsonObject(
+        Maps.immutableMap("a.b(0).c", "hello", "a.b(1).c", "hello")),
+        new JSONObject("{a:{b:[{c:\"hello\"},{c:\"hello\"}]}}"));
+    assertJsonEquals(JsonConversionUtils.parametersToJsonObject(
+        Maps.immutableMap("a.b.c", "hello, true, false, null, 1,2, \"null\", \"()\"")),
+        new JSONObject("{a:{b:{c:[\"hello\",true,false,null,1,2,\"null\",\"()\"]}}}"));
+    assertJsonEquals(JsonConversionUtils.parametersToJsonObject(
+        Maps.immutableMap("a.b.c", "\"hello, true, false, null, 1,2\"")),
+        new JSONObject("{a:{b:{c:\"hello, true, false, null, 1,2\"}}}"));
+  }
+
+  public void testJSONToParameterMapParsing()
+      throws Exception {
+    java.util.Map resultMap = JsonConversionUtils
+        .fromJson(new JSONObject("{a:{b:[{c:\"hello\"},{c:\"hello\"}]}}"));
+  }
+
+  private void assertJsonEquals(Object expected, Object actual)
+      throws JSONException {
+    if (expected == null) {
+      assertNull(actual);
+      return;
+    }
+    assertNotNull(actual);
+    if (expected instanceof JSONObject) {
+      JSONObject expectedObject = (JSONObject) expected;
+      JSONObject actualObject = (JSONObject) actual;
+      if (expectedObject.length() == 0) {
+        assertEquals(expectedObject.length(), actualObject.length());
+        return;
+      }
+      assertEquals(expectedObject.names().length(), actualObject.names().length());
+      String key;
+      for (Iterator keys = expectedObject.keys(); keys.hasNext();
+          assertJsonEquals(expectedObject.get(key), actualObject.get(key))) {
+        key = (String) keys.next();
+        assertTrue(actualObject.has(key));
+      }
+    } else if (expected instanceof JSONArray) {
+      JSONArray expectedArray = (JSONArray) expected;
+      JSONArray actualArray = (JSONArray) actual;
+      assertEquals(expectedArray.length(), actualArray.length());
+      for (int i = 0; i < expectedArray.length(); i++) {
+        if (expectedArray.isNull(i)) {
+          assertTrue(actualArray.isNull(i));
+        } else {
+          assertJsonEquals(expectedArray.get(i), actualArray.get(i));
+        }
+      }
+
+    } else {
+      assertEquals(expected, actual);
+    }
+  }
+}

Added: incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/JsonRpcServletTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/JsonRpcServletTest.java?rev=687235&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/JsonRpcServletTest.java (added)
+++ incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/JsonRpcServletTest.java Tue Aug 19 21:14:39 2008
@@ -0,0 +1,283 @@
+/*
+ * 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.shindig.social.opensocial.service;
+
+import org.apache.shindig.common.SecurityTokenException;
+import org.apache.shindig.common.uri.Uri;
+import org.apache.shindig.common.testing.FakeGadgetToken;
+import org.apache.shindig.common.util.ImmediateFuture;
+import org.apache.shindig.social.ResponseItem;
+import org.apache.shindig.social.core.oauth.AuthenticationServletFilter;
+import org.apache.shindig.social.core.util.BeanJsonConverter;
+import org.apache.shindig.social.core.util.BeanXmlConverter;
+
+import com.google.inject.Injector;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+
+import org.easymock.classextension.EasyMock;
+import org.json.JSONObject;
+import org.json.JSONArray;
+
+import junit.framework.TestCase;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.Map;
+
+import javax.servlet.http.HttpServletResponse;
+
+/**
+ *
+ */
+public class JsonRpcServletTest extends TestCase {
+
+  private static final FakeGadgetToken FAKE_GADGET_TOKEN = new FakeGadgetToken()
+      .setOwnerId("john.doe").setViewerId("john.doe");
+
+  private AuthenticationServletFilter.SecurityTokenRequest req;
+
+  private HttpServletResponse res;
+
+  private JsonRpcServlet servlet;
+
+  private PersonHandler peopleHandler;
+
+  private ActivityHandler activityHandler;
+
+  private AppDataHandler appDataHandler;
+
+  private Injector injector;
+
+  private BeanJsonConverter jsonConverter;
+
+  private BeanXmlConverter xmlConverter;
+
+  protected void setUp() throws Exception {
+    servlet = new JsonRpcServlet();
+    req = EasyMock.createMock(AuthenticationServletFilter.SecurityTokenRequest.class);
+    res = EasyMock.createMock(HttpServletResponse.class);
+    jsonConverter = EasyMock.createMock(BeanJsonConverter.class);
+    xmlConverter = EasyMock.createMock(BeanXmlConverter.class);
+
+    peopleHandler = EasyMock.createMock(PersonHandler.class);
+    activityHandler = EasyMock.createMock(ActivityHandler.class);
+    appDataHandler = EasyMock.createMock(AppDataHandler.class);
+
+    injector = EasyMock.createMock(Injector.class);
+    servlet.setInjector(injector);
+
+    servlet.setHandlers(new HandlerProvider(new PersonHandler(null), new ActivityHandler(null),
+        new AppDataHandler(null)));
+
+    servlet.setBeanConverters(jsonConverter, xmlConverter);
+  }
+
+  private void setupInjector() {
+    EasyMock.expect(injector.getInstance(PersonHandler.class)).andStubReturn(peopleHandler);
+    EasyMock.expect(injector.getInstance(ActivityHandler.class)).andStubReturn(activityHandler);
+    EasyMock.expect(injector.getInstance(AppDataHandler.class)).andStubReturn(appDataHandler);
+  }
+
+  public void testPeopleMethodRecognition() throws Exception {
+    verifyHandlerWasFoundForMethod("{method:people.get,id:id,params:{userId:5,groupId:@self}}",
+        peopleHandler);
+  }
+
+  public void testActivitiesMethodRecognition() throws Exception {
+    verifyHandlerWasFoundForMethod("{method:activities.get,id:id,params:{userId:5,groupId:@self}}",
+        activityHandler);
+  }
+
+  public void testAppDataMethodRecognition() throws Exception {
+    verifyHandlerWasFoundForMethod("{method:appdata.get,id:id,params:{userId:5,groupId:@self}}",
+        appDataHandler);
+  }
+
+  public void testInvalidService() throws Exception {
+    String json = "{method:junk.get,id:id,params:{userId:5,groupId:@self}}";
+    setupRequest(json);
+    setupInjector();
+
+    JSONObject err = new JSONObject(
+        "{id:id,error:{message:'The service junk is not implemented',code:501}}");
+
+    PrintWriter writerMock = EasyMock.createMock(PrintWriter.class);
+    EasyMock.expect(res.getWriter()).andReturn(writerMock);
+    writerMock.write(EasyMock.eq(err.toString()));
+    EasyMock.expectLastCall();
+
+    EasyMock.replay(req, res, injector, jsonConverter, writerMock);
+    servlet.service(req, res);
+    EasyMock.verify(req, res, injector, jsonConverter, writerMock);
+    EasyMock.reset(req, res, injector, jsonConverter);
+  }
+
+
+  /**
+   * Tests a data handler that returns a failed Future
+   */
+  public void testFailedRequest() throws Exception {
+    setupRequest("{id:id,method:appdata.get}");
+    EasyMock.expect(injector.getInstance(AppDataHandler.class)).andStubReturn(appDataHandler);
+    setupInjector();
+
+    EasyMock.expect(appDataHandler.handleItem(EasyMock.isA(RpcRequestItem.class)));
+    EasyMock.expectLastCall().andReturn(new FailingFuture());
+
+    JSONObject err = new JSONObject(
+        "{id:id,error:{message:'FAILED',code:500}}");
+
+    PrintWriter writerMock = EasyMock.createMock(PrintWriter.class);
+    EasyMock.expect(res.getWriter()).andReturn(writerMock);
+    writerMock.write(EasyMock.eq(err.toString()));
+    EasyMock.expectLastCall();
+
+    EasyMock.replay(req, res, appDataHandler, injector, jsonConverter, writerMock);
+    servlet.service(req, res);
+    EasyMock.verify(req, res, appDataHandler, injector, jsonConverter, writerMock);
+    EasyMock.reset(req, res, appDataHandler, injector, jsonConverter);
+  }
+
+  private void verifyHandlerWasFoundForMethod(String json, DataRequestHandler handler)
+      throws Exception {
+    setupRequest(json);
+    setupInjector();
+
+    String resultObject = "my lovely json";
+    ResponseItem<String> response = new ResponseItem<String>(resultObject);
+
+    EasyMock.expect(handler.handleItem(EasyMock.isA(RequestItem.class)));
+    EasyMock.expectLastCall().andReturn(ImmediateFuture.newInstance(response));
+
+    EasyMock.expect(jsonConverter.convertToJson(resultObject)).andReturn(resultObject);
+
+    JSONObject result = new JSONObject();
+    result.put("id", "id");
+    result.put("data", resultObject);
+    PrintWriter writerMock = EasyMock.createMock(PrintWriter.class);
+    EasyMock.expect(res.getWriter()).andReturn(writerMock);
+    writerMock.write(EasyMock.eq(result.toString()));
+    EasyMock.expectLastCall();
+
+    EasyMock.replay(req, res, handler, injector, jsonConverter, writerMock);
+    servlet.service(req, res);
+    EasyMock.verify(req, res, handler, injector, jsonConverter, writerMock);
+    EasyMock.reset(req, res, handler, injector, jsonConverter);
+  }
+
+  public void testBasicBatch() throws Exception {
+    String batchJson =
+        "[{method:people.get,id:'1'},{method:activities.get,id:'2'}]";
+    setupRequest(batchJson);
+    setupInjector();
+
+    String resultObject = "my lovely json";
+    ResponseItem response = new ResponseItem<String>(resultObject);
+
+    Future<? extends ResponseItem> responseItemFuture = ImmediateFuture.newInstance(response);
+    EasyMock.expect(peopleHandler.handleItem(EasyMock.isA(RequestItem.class)));
+    EasyMock.expectLastCall().andReturn(responseItemFuture);
+    EasyMock.expect(activityHandler.handleItem(EasyMock.isA(RequestItem.class)));
+    EasyMock.expectLastCall().andReturn(responseItemFuture);
+
+    EasyMock.expect(jsonConverter.convertToJson(resultObject)).andReturn(resultObject);
+    EasyMock.expect(jsonConverter.convertToJson(resultObject)).andReturn(resultObject);
+
+    JSONArray result = new JSONArray("[{id:'1',data:'my lovely json'}," +
+        "{id:'2',data:my lovely json}]");
+    PrintWriter writerMock = EasyMock.createMock(PrintWriter.class);
+    EasyMock.expect(res.getWriter()).andReturn(writerMock);
+    writerMock.write(EasyMock.eq(result.toString()));
+    EasyMock.expectLastCall();
+
+    EasyMock.replay(req, res, peopleHandler, activityHandler, injector, jsonConverter, writerMock);
+    servlet.service(req, res);
+    EasyMock.verify(req, res, peopleHandler, activityHandler, injector, jsonConverter, writerMock);
+    EasyMock.reset(req, res, peopleHandler, activityHandler, injector, jsonConverter);
+  }
+
+  public void testGetExecution() throws Exception {
+    EasyMock.expect(req.getParameterMap()).andStubReturn(
+        Maps.immutableMap("method",new String[]{"people.get"},"id", new String[]{"1"}));
+    EasyMock.expect(req.getMethod()).andStubReturn("GET");
+    EasyMock.expect(req.getToken()).andReturn(FAKE_GADGET_TOKEN);
+    setupInjector();
+
+    String resultObject = "my lovely json";
+    ResponseItem response = new ResponseItem<String>(resultObject);
+
+    Future<? extends ResponseItem> responseItemFuture = ImmediateFuture.newInstance(response);
+    EasyMock.expect(peopleHandler.handleItem(EasyMock.isA(RequestItem.class)));
+    EasyMock.expectLastCall().andReturn(responseItemFuture);
+
+    EasyMock.expect(jsonConverter.convertToJson(resultObject)).andReturn(resultObject);
+
+    JSONObject result = new JSONObject("{id:'1',data:'my lovely json'}");
+    PrintWriter writerMock = EasyMock.createMock(PrintWriter.class);
+    EasyMock.expect(res.getWriter()).andReturn(writerMock);
+    writerMock.write(EasyMock.eq(result.toString()));
+    EasyMock.expectLastCall();
+
+    EasyMock.replay(req, res, peopleHandler, activityHandler, injector, jsonConverter, writerMock);
+    servlet.service(req, res);
+    EasyMock.verify(req, res, peopleHandler, activityHandler, injector, jsonConverter, writerMock);
+    EasyMock.reset(req, res, peopleHandler, activityHandler, injector, jsonConverter);
+  }
+
+  private void setupRequest(String json)
+      throws IOException, SecurityTokenException {
+    EasyMock.expect(req.getReader()).andStubReturn(new BufferedReader(new StringReader(json)));
+    EasyMock.expect(req.getMethod()).andStubReturn("POST");
+    EasyMock.expect(req.getToken()).andReturn(FAKE_GADGET_TOKEN);
+  }
+
+  /**
+   * Future implementation that fails with an exception.
+   */
+  private static class FailingFuture implements Future<ResponseItem> {
+
+    public boolean cancel(boolean mayInterruptIfRunning) {
+      return false;
+    }
+
+    public boolean isCancelled() {
+      return false;
+    }
+
+    public boolean isDone() {
+      return true;
+    }
+
+    public ResponseItem get() throws InterruptedException, ExecutionException {
+      throw new ExecutionException(new RuntimeException("FAILED"));
+    }
+
+    public ResponseItem get(long timeout, TimeUnit unit)
+        throws InterruptedException, ExecutionException, TimeoutException {
+      return null;
+    }
+  }
+}
+

Modified: incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/PersonHandlerTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/PersonHandlerTest.java?rev=687235&r1=687234&r2=687235&view=diff
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/PersonHandlerTest.java (original)
+++ incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/PersonHandlerTest.java Tue Aug 19 21:14:39 2008
@@ -45,7 +45,7 @@
 
   private FakeGadgetToken token;
 
-  private RequestItem request;
+  private RestfulRequestItem request;
 
   private static final Set<String> DEFAULT_FIELDS = Sets.newHashSet(Person.Field.ID.toString(),
       Person.Field.NAME.toString(),
@@ -81,12 +81,10 @@
   }
 
   private void setPathAndParams(String path, Map<String, String> params) {
-    request = new RequestItem();
-    request.setUrl(path);
+    request = new RestfulRequestItem(path, "GET", null, token, null);
     for (Map.Entry<String, String> entry : params.entrySet()) {
       request.setParameter(entry.getKey(), entry.getValue());
     }
-    request.setToken(token);
   }
 
   public void testHandleGetAllNoParams() throws Exception {
@@ -214,4 +212,4 @@
     assertEquals(ResponseError.NOT_IMPLEMENTED, handler.handlePost(request).get().getError());
     verify();
   }
-}
\ No newline at end of file
+}

Added: incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/RestfulRequestItemTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/RestfulRequestItemTest.java?rev=687235&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/RestfulRequestItemTest.java (added)
+++ incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/RestfulRequestItemTest.java Tue Aug 19 21:14:39 2008
@@ -0,0 +1,135 @@
+/*
+ * 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.shindig.social.opensocial.service;
+
+import org.apache.shindig.common.testing.FakeGadgetToken;
+import org.apache.shindig.social.core.oauth.AuthenticationServletFilter;
+import org.apache.shindig.social.opensocial.spi.GroupId;
+import org.apache.shindig.social.opensocial.spi.PersonService;
+import org.apache.shindig.social.opensocial.spi.UserId;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import org.easymock.classextension.EasyMock;
+
+import junit.framework.TestCase;
+
+import java.util.Collections;
+
+public class RestfulRequestItemTest extends TestCase {
+
+  private static final FakeGadgetToken FAKE_TOKEN = new FakeGadgetToken();
+
+  private static final String DEFAULT_PATH = "/people/john.doe/@self";
+
+  private RestfulRequestItem request;
+
+  protected void setUp() throws Exception {
+    super.setUp();
+    request = new RestfulRequestItem(
+        DEFAULT_PATH + "?fields=huey,dewey,louie", "GET",
+        null, FAKE_TOKEN, null);
+  }
+
+  public void testParseUrl() throws Exception {
+    assertEquals("people", request.getService());
+    assertEquals(Lists.newArrayList("huey", "dewey", "louie"), request.getListParameter("fields"));
+
+    // Try it without any params
+    request = new RestfulRequestItem(DEFAULT_PATH, "GET", null, null, null);
+
+    assertEquals("people", request.getService());
+    assertEquals(null, request.getParameters().get("fields"));
+  }
+
+  public void testGetHttpMethodFromParameter() throws Exception {
+    AuthenticationServletFilter.SecurityTokenRequest overridden =
+        EasyMock.createMock(AuthenticationServletFilter.SecurityTokenRequest.class);
+    EasyMock.expect(overridden.getParameter(RestfulRequestItem.X_HTTP_METHOD_OVERRIDE))
+        .andReturn("DELETE");
+    EasyMock.replay(overridden);
+    assertEquals("DELETE", RestfulRequestItem.getMethod(overridden));
+    EasyMock.verify(overridden);
+  }
+
+
+  public void testGetAppId() throws Exception {
+    request.setParameter("appId", "100");
+    assertEquals("100", request.getAppId());
+
+    request.setParameter("appId", "@app");
+    assertEquals(FAKE_TOKEN.getAppId(), request.getAppId());
+  }
+
+  public void testGetUser() throws Exception {
+    request.setParameter("userId", "@owner");
+    assertEquals(UserId.Type.owner, request.getUsers().iterator().next().getType());
+  }
+
+  public void testGetGroup() throws Exception {
+    request.setParameter("groupId", "@self");
+    assertEquals(GroupId.Type.self, request.getGroup().getType());
+  }
+
+  public void testStartIndex() throws Exception {
+    request.setParameter("startIndex", null);
+    assertEquals(0, request.getStartIndex());
+
+    request.setParameter("startIndex", "5");
+    assertEquals(5, request.getStartIndex());
+  }
+
+  public void testCount() throws Exception {
+    request.setParameter("count", null);
+    assertEquals(20, request.getCount());
+
+    request.setParameter("count", "5");
+    assertEquals(5, request.getCount());
+  }
+
+  public void testOrderBy() throws Exception {
+    request.setParameter("orderBy", null);
+    assertEquals(PersonService.SortOrder.topFriends, request.getOrderBy());
+
+    request.setParameter("orderBy", "name");
+    assertEquals(PersonService.SortOrder.name, request.getOrderBy());
+  }
+
+  public void testFilterBy() throws Exception {
+    request.setParameter("filterBy", null);
+    assertEquals(PersonService.FilterType.all, request.getFilterBy());
+
+    request.setParameter("filterBy", "hasApp");
+    assertEquals(PersonService.FilterType.hasApp, request.getFilterBy());
+  }
+
+  public void testFields() throws Exception {
+    request.setListParameter("fields", Collections.<String>emptyList());
+    assertEquals(Sets.<String>newHashSet(), request.getFields());
+
+    request.setParameter("fields", "happy,sad,grumpy");
+    assertEquals(Sets.newHashSet("happy", "sad", "grumpy"), request.getFields());
+  }
+
+  public void testRouteFromParameter() throws Exception {
+    assertEquals("path", RestfulRequestItem.getServiceFromPath("/path"));
+    assertEquals("path", RestfulRequestItem.getServiceFromPath("/path/fun"));
+    assertEquals("path", RestfulRequestItem.getServiceFromPath("/path/fun/yes"));
+  }
+}

Added: incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/RpcRequestItemTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/RpcRequestItemTest.java?rev=687235&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/RpcRequestItemTest.java (added)
+++ incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/service/RpcRequestItemTest.java Tue Aug 19 21:14:39 2008
@@ -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.shindig.social.opensocial.service;
+
+import org.apache.shindig.common.testing.FakeGadgetToken;
+import org.apache.shindig.social.opensocial.spi.GroupId;
+import org.apache.shindig.social.opensocial.spi.PersonService;
+import org.apache.shindig.social.opensocial.spi.UserId;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import org.json.JSONObject;
+
+import junit.framework.TestCase;
+
+import java.util.Collections;
+
+public class RpcRequestItemTest extends TestCase {
+
+  private static final FakeGadgetToken FAKE_TOKEN = new FakeGadgetToken();
+
+
+  private RpcRequestItem request;
+
+  private JSONObject baseRpc;
+
+  protected void setUp() throws Exception {
+    super.setUp();
+    baseRpc = new JSONObject(
+        "{method:people.get,id:id,params:{"
+            + "userId:john.doe,"
+            + "groupId:@self,"
+            + "fields:[huey,dewey,louie]"
+            + "}}");
+    request = new RpcRequestItem(baseRpc, FAKE_TOKEN, null);
+  }
+
+  public void testParseMethod() throws Exception {
+    assertEquals("people", request.getService());
+    assertEquals(Lists.newArrayList("huey", "dewey", "louie"), request.getListParameter("fields"));
+
+    // Try it without any params
+    JSONObject noParams = new JSONObject(baseRpc.toString());
+    noParams.remove("params");
+    request = new RpcRequestItem(noParams, FAKE_TOKEN, null);
+
+    assertEquals("people", request.getService());
+    assertEquals(Collections.<String>emptyList(), request.getListParameter("fields"));
+  }
+
+  public void testGetAppId() throws Exception {
+    request.setParameter("appId", "100");
+    assertEquals("100", request.getAppId());
+
+    request.setParameter("appId", "@app");
+    assertEquals(FAKE_TOKEN.getAppId(), request.getAppId());
+  }
+
+  public void testGetUser() throws Exception {
+    request.setParameter("userId", "@owner");
+    assertEquals(UserId.Type.owner, request.getUsers().iterator().next().getType());
+  }
+
+  public void testGetGroup() throws Exception {
+    request.setParameter("groupId", "@self");
+    assertEquals(GroupId.Type.self, request.getGroup().getType());
+  }
+
+  public void testStartIndex() throws Exception {
+    request.setParameter("startIndex", null);
+    assertEquals(0, request.getStartIndex());
+
+    request.setParameter("startIndex", "5");
+    assertEquals(5, request.getStartIndex());
+  }
+
+  public void testCount() throws Exception {
+    request.setParameter("count", null);
+    assertEquals(20, request.getCount());
+
+    request.setParameter("count", "5");
+    assertEquals(5, request.getCount());
+  }
+
+  public void testOrderBy() throws Exception {
+    request.setParameter("orderBy", null);
+    assertEquals(PersonService.SortOrder.topFriends, request.getOrderBy());
+
+    request.setParameter("orderBy", "name");
+    assertEquals(PersonService.SortOrder.name, request.getOrderBy());
+  }
+
+  public void testFilterBy() throws Exception {
+    request.setParameter("filterBy", null);
+    assertEquals(PersonService.FilterType.all, request.getFilterBy());
+
+    request.setParameter("filterBy", "hasApp");
+    assertEquals(PersonService.FilterType.hasApp, request.getFilterBy());
+  }
+
+  public void testFields() throws Exception {
+    request.setListParameter("fields", Collections.<String>emptyList());
+    assertEquals(Sets.<String>newHashSet(), request.getFields());
+
+    request.setListParameter("fields", Lists.newArrayList("happy","sad","grumpy"));
+    assertEquals(Sets.newHashSet("happy", "sad", "grumpy"), request.getFields());
+  }
+}

Modified: incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/spi/DataRequestHandlerTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/spi/DataRequestHandlerTest.java?rev=687235&r1=687234&r2=687235&view=diff
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/spi/DataRequestHandlerTest.java (original)
+++ incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/opensocial/spi/DataRequestHandlerTest.java Tue Aug 19 21:14:39 2008
@@ -22,6 +22,7 @@
 import org.apache.shindig.social.ResponseItem;
 import org.apache.shindig.social.opensocial.service.DataRequestHandler;
 import org.apache.shindig.social.opensocial.service.RequestItem;
+import org.apache.shindig.social.opensocial.service.RestfulRequestItem;
 
 import junit.framework.TestCase;
 
@@ -31,8 +32,6 @@
 
   private DataRequestHandler drh;
 
-  private RequestItem request;
-
   @Override
   protected void setUp() throws Exception {
     drh = new DataRequestHandler() {
@@ -52,8 +51,6 @@
         return ImmediateFuture.newInstance(new ResponseItem<String>("GET"));
       }
     };
-
-    request = new RequestItem();
   }
 
   public void testHandleItemSuccess() throws Exception {
@@ -64,7 +61,7 @@
   }
 
   private void verifyItemDispatchMethodCalled(String methodName) throws Exception {
-    request.setMethod(methodName);
+    RestfulRequestItem request = new RestfulRequestItem(null, methodName, null, null);
     assertEquals(methodName, drh.handleItem(request).get().getResponse());
   }
 
@@ -76,7 +73,7 @@
   }
 
   private void verifyDispatchMethodCalled(String methodName) throws Exception {
-    request.setMethod(methodName);
+    RestfulRequestItem request = new RestfulRequestItem(null, methodName, null, null);
     assertEquals(methodName, drh.handleItem(request).get().getResponse());
   }
 
@@ -87,8 +84,8 @@
   }
 
   private void verifyExceptionThrown(String methodName) throws Exception {
-    request.setMethod(methodName);
+    RestfulRequestItem request = new RestfulRequestItem(null, methodName, null, null);
     Future<? extends ResponseItem> err = drh.handleItem(request);
     assertEquals(err.get().getError(), ResponseError.NOT_IMPLEMENTED);
   }
-}
\ No newline at end of file
+}