You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@shindig.apache.org by do...@apache.org on 2008/06/12 03:07:29 UTC

svn commit: r666940 - in /incubator/shindig/trunk/java: server/src/main/webapp/WEB-INF/ social-api/src/main/java/org/apache/shindig/social/ social-api/src/main/java/org/apache/shindig/social/dataservice/ social-api/src/test/java/org/apache/shindig/soci...

Author: doll
Date: Wed Jun 11 18:07:28 2008
New Revision: 666940

URL: http://svn.apache.org/viewvc?rev=666940&view=rev
Log:
A new trial implementation of the json restful wire format. This new code lives solely in the dataservice package. It includes 3 handlers for people, data and activities which currently support gets, posts and deletes. Pagination, partial updates and so forth should all be working. 

The next steps for this format are to 
- write more tests. (Hopefully re-using some of the abdera end to end tests that we have)
- implement batching
- fix any other non-compliant things

If you use the samplecontainer with useRestful=true and path=/social/jsonrest you will see the new code in action. Both of the checked in sample gadgets work. 

More patches and code coming tomorrow... 



Added:
    incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/ActivityHandler.java
    incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/AppDataHandler.java
    incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/DataRequestHandler.java
    incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/DataServiceServlet.java
    incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/PersonHandler.java
    incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/dataservice/
    incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/dataservice/DataRequestHandlerTest.java
    incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/dataservice/DataServiceServletTest.java
Modified:
    incubator/shindig/trunk/java/server/src/main/webapp/WEB-INF/web.social.xml
    incubator/shindig/trunk/java/server/src/main/webapp/WEB-INF/web.xml
    incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/SocialApiGuiceModule.java

Modified: incubator/shindig/trunk/java/server/src/main/webapp/WEB-INF/web.social.xml
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/server/src/main/webapp/WEB-INF/web.social.xml?rev=666940&r1=666939&r2=666940&view=diff
==============================================================================
--- incubator/shindig/trunk/java/server/src/main/webapp/WEB-INF/web.social.xml (original)
+++ incubator/shindig/trunk/java/server/src/main/webapp/WEB-INF/web.social.xml Wed Jun 11 18:07:28 2008
@@ -41,6 +41,14 @@
     </servlet-class>
   </servlet>
 
+  <!-- Serve the new rest code -->
+  <servlet>
+    <servlet-name>jsonrest</servlet-name>
+    <servlet-class>
+      org.apache.shindig.social.dataservice.DataServiceServlet
+    </servlet-class>
+  </servlet>
+
   <!-- Serve REST api -->
   <servlet>
     <servlet-name>restapiServlet</servlet-name>
@@ -59,6 +67,11 @@
   </servlet-mapping>
 
   <servlet-mapping>
+    <servlet-name>jsonrest</servlet-name>
+    <url-pattern>/social/jsonrest/*</url-pattern>
+  </servlet-mapping>
+
+  <servlet-mapping>
     <servlet-name>restapiServlet</servlet-name>
     <url-pattern>/social/rest/*</url-pattern>
   </servlet-mapping>

Modified: incubator/shindig/trunk/java/server/src/main/webapp/WEB-INF/web.xml
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/server/src/main/webapp/WEB-INF/web.xml?rev=666940&r1=666939&r2=666940&view=diff
==============================================================================
--- incubator/shindig/trunk/java/server/src/main/webapp/WEB-INF/web.xml (original)
+++ incubator/shindig/trunk/java/server/src/main/webapp/WEB-INF/web.xml Wed Jun 11 18:07:28 2008
@@ -53,7 +53,7 @@
   <servlet>
     <servlet-name>makeRequest</servlet-name>
     <servlet-class>
-      org.apache.shindig.gadgets.MakeRequestServlet
+      org.apache.shindig.gadgets.servlet.MakeRequestServlet
     </servlet-class>
   </servlet>
 
@@ -81,6 +81,14 @@
     </servlet-class>
   </servlet>
 
+  <!-- Serve the new rest code -->
+  <servlet>
+    <servlet-name>jsonrest</servlet-name>
+    <servlet-class>
+      org.apache.shindig.social.dataservice.DataServiceServlet
+    </servlet-class>
+  </servlet>
+
   <!-- javascript serving -->
   <servlet>
     <servlet-name>js</servlet-name>
@@ -135,6 +143,11 @@
   </servlet-mapping>
 
   <servlet-mapping>
+    <servlet-name>jsonrest</servlet-name>
+    <url-pattern>/social/jsonrest/*</url-pattern>
+  </servlet-mapping>
+
+  <servlet-mapping>
     <servlet-name>restapiServlet</servlet-name>
     <url-pattern>/social/rest/*</url-pattern>
   </servlet-mapping>

Modified: incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/SocialApiGuiceModule.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/SocialApiGuiceModule.java?rev=666940&r1=666939&r2=666940&view=diff
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/SocialApiGuiceModule.java (original)
+++ incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/SocialApiGuiceModule.java Wed Jun 11 18:07:28 2008
@@ -28,14 +28,17 @@
 import org.apache.shindig.social.samplecontainer.BasicPeopleService;
 import org.apache.shindig.social.samplecontainer.SampleContainerRouteManager;
 import org.apache.shindig.social.samplecontainer.StateFileDataHandler;
+import org.apache.shindig.social.dataservice.*;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.google.inject.AbstractModule;
 import com.google.inject.Inject;
 import com.google.inject.Provider;
 import com.google.inject.TypeLiteral;
 
 import java.util.List;
+import java.util.Map;
 
 /**
  * Provides social api component injection
@@ -51,6 +54,11 @@
 
     bind(new TypeLiteral<List<GadgetDataHandler>>() {})
         .toProvider(GadgetDataHandlersProvider.class);
+    bind(new TypeLiteral<Map<String, DataRequestHandler>>() {})
+        .toProvider(DataRequestHandlersProvider.class);
+    bind(PersonService.class).to(BasicPeopleService.class);
+    bind(ActivityService.class).to(BasicActivitiesService.class);
+    bind(AppDataService.class).to(BasicDataService.class);
 
     bind(SocialRouteManager.class).to(SampleContainerRouteManager.class);
   }
@@ -70,4 +78,21 @@
     }
   }
 
+  public static class DataRequestHandlersProvider
+      implements Provider<Map<String, DataRequestHandler>> {
+    Map<String, DataRequestHandler> handlers;
+
+    @Inject
+    public DataRequestHandlersProvider(PersonHandler peopleHandler, ActivityHandler activityHandler, AppDataHandler appDataHandler) {
+      handlers = Maps.newHashMap();
+      handlers.put(DataServiceServlet.PEOPLE_ROUTE, peopleHandler);
+      handlers.put(DataServiceServlet.ACTIVITY_ROUTE, activityHandler);
+      handlers.put(DataServiceServlet.APPDATA_ROUTE, appDataHandler);
+    }
+
+    public Map<String, DataRequestHandler> get() {
+      return handlers;
+    }
+  }
+
 }

Added: incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/ActivityHandler.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/ActivityHandler.java?rev=666940&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/ActivityHandler.java (added)
+++ incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/ActivityHandler.java Wed Jun 11 18:07:28 2008
@@ -0,0 +1,108 @@
+/*
+ * 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.dataservice;
+
+import com.google.inject.Inject;
+import org.apache.shindig.common.SecurityToken;
+import org.apache.shindig.social.ResponseError;
+import org.apache.shindig.social.ResponseItem;
+import org.apache.shindig.social.opensocial.model.Activity;
+import org.apache.shindig.social.opensocial.util.BeanJsonConverter;
+
+import javax.servlet.http.HttpServletRequest;
+
+public class ActivityHandler extends DataRequestHandler {
+  private ActivityService service;
+
+  @Inject
+  public ActivityHandler(ActivityService service, BeanJsonConverter converter) {
+    super(converter);
+    this.service = service;
+  }
+
+  ResponseItem handleDelete(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    return new ResponseItem<Object>(ResponseError.BAD_REQUEST,
+        "You can't delete activities. ", null);
+  }
+
+  /**
+   * /activities/{userId}/@self
+   *
+   * examples:
+   * /activities/john.doe/@self
+   * - postBody is an activity object
+   */
+  ResponseItem handlePut(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    return handlePost(servletRequest, token);
+  }
+
+  /**
+   * /activities/{userId}/@self
+   *
+   * examples:
+   * /activities/john.doe/@self
+   * - postBody is an activity object
+   */
+  ResponseItem handlePost(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    String[] segments = getParamsFromRequest(servletRequest);
+
+    String userId = segments[0];
+    DataServiceServlet.GroupId groupId
+        = DataServiceServlet.GroupId.fromJson(segments[1]);
+    // TODO: Should we pass the groupId through to the service?
+
+    String jsonActivity = servletRequest.getParameter("entry");
+    Activity activity = converter.convertToObject(jsonActivity, Activity.class);
+
+    return service.createActivity(userId, activity, token);
+  }
+
+  /**
+   * /activities/{userId}/{groupId}/{optionalActvityId}
+   *
+   * examples:
+   * /activities/john.doe/@self/1
+   * /activities/john.doe/@self
+   * /activities/john.doe/@friends
+   */
+  ResponseItem handleGet(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    String[] segments = getParamsFromRequest(servletRequest);
+
+    String userId = segments[0];
+    DataServiceServlet.GroupId groupId
+        = DataServiceServlet.GroupId.fromJson(segments[1]);
+    String optionalActivityId = null;
+    if (segments.length > 2) {
+      optionalActivityId = segments[2];
+    }
+
+    // TODO: Filter by fields
+    // TODO: do we need to add pagination and sorting support?
+    if (optionalActivityId != null) {
+      return service.getActivity(userId, groupId, optionalActivityId, token);
+    }
+    return service.getActivities(userId, groupId, token);
+  }
+
+}
+
+

Added: incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/AppDataHandler.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/AppDataHandler.java?rev=666940&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/AppDataHandler.java (added)
+++ incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/AppDataHandler.java Wed Jun 11 18:07:28 2008
@@ -0,0 +1,147 @@
+/*
+ * 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.dataservice;
+
+import org.apache.shindig.common.SecurityToken;
+import org.apache.shindig.social.ResponseItem;
+import org.apache.shindig.social.opensocial.util.BeanJsonConverter;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.inject.Inject;
+import com.google.inject.TypeLiteral;
+
+import javax.servlet.http.HttpServletRequest;
+import java.util.List;
+import java.util.Map;
+
+import org.json.JSONObject;
+import org.json.JSONException;
+
+public class AppDataHandler extends DataRequestHandler {
+  private AppDataService service;
+
+  @Inject
+  public AppDataHandler(AppDataService service, BeanJsonConverter converter) {
+    super(converter);
+    this.service = service;
+  }
+
+  /**
+   * /people/{userId}/{groupId}/{appId}
+   * - fields={field1, field2}
+   *
+   * examples:
+   * /appdata/john.doe/@friends/app?fields=count
+   * /appdata/john.doe/@self/app
+   *
+   * The post data should be a regular json object. All of the fields vars will
+   * be pulled from the values and set on the person object. If there are no
+   * fields vars then all of the data will be overridden.
+   */
+  ResponseItem handleDelete(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    String[] segments = getParamsFromRequest(servletRequest);
+
+    String userId = segments[0];
+    DataServiceServlet.GroupId groupId
+        = DataServiceServlet.GroupId.fromJson(segments[1]);
+    String appId = segments[2];
+
+    List<String> fields = getListParam(servletRequest, "fields",
+        Lists.<String>newArrayList());
+
+    return service.deletePersonData(userId, groupId, fields,
+        appId, token);
+  }
+
+  /**
+   * /people/{userId}/{groupId}/{appId}
+   * - fields={field1, field2}
+   *
+   * examples:
+   * /appdata/john.doe/@friends/app?fields=count
+   * /appdata/john.doe/@self/app
+   *
+   * The post data should be a regular json object. All of the fields vars will
+   * be pulled from the values and set on the person object. If there are no
+   * fields vars then all of the data will be overridden.
+   */
+  ResponseItem handlePut(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    return handlePost(servletRequest, token);
+  }
+
+  /**
+   * /people/{userId}/{groupId}/{appId}
+   * - fields={field1, field2}
+   *
+   * examples:
+   * /appdata/john.doe/@friends/app?fields=count
+   * /appdata/john.doe/@self/app
+   *
+   * The post data should be a regular json object. All of the fields vars will
+   * be pulled from the values and set on the person object. If there are no
+   * fields vars then all of the data will be overridden.
+   */
+  ResponseItem handlePost(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    String[] segments = getParamsFromRequest(servletRequest);
+
+    String userId = segments[0];
+    DataServiceServlet.GroupId groupId
+        = DataServiceServlet.GroupId.fromJson(segments[1]);
+    String appId = segments[2];
+
+    List<String> fields = getListParam(servletRequest, "fields",
+        Lists.<String>newArrayList());
+
+    String jsonAppData = servletRequest.getParameter("entry");
+    Map<String, String> values = Maps.newHashMap();
+    values = converter.convertToObject(jsonAppData.toString(),
+        (Class<Map<String, String>>) values.getClass());
+
+    return service.updatePersonData(userId, groupId, fields, values,
+        appId, token);
+  }
+
+  /**
+   * /appdata/{userId}/{groupId}/{appId}
+   * - fields={field1, field2}
+   *
+   * examples:
+   * /appdata/john.doe/@friends/app?fields=count
+   * /appdata/john.doe/@self/app
+   */
+  ResponseItem handleGet(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    String[] segments = getParamsFromRequest(servletRequest);
+
+    String userId = segments[0];
+    DataServiceServlet.GroupId groupId
+        = DataServiceServlet.GroupId.fromJson(segments[1]);
+    String appId = segments[2];
+
+    List<String> fields = getListParam(servletRequest, "fields",
+        Lists.<String>newArrayList());
+
+    return service.getPersonData(userId, groupId, fields, appId, token);
+  }
+
+}
+

Added: incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/DataRequestHandler.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/DataRequestHandler.java?rev=666940&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/DataRequestHandler.java (added)
+++ incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/DataRequestHandler.java Wed Jun 11 18:07:28 2008
@@ -0,0 +1,123 @@
+/*
+ * 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.dataservice;
+
+import org.apache.shindig.common.SecurityToken;
+import org.apache.shindig.social.ResponseItem;
+import org.apache.shindig.social.opensocial.util.BeanJsonConverter;
+
+import com.google.inject.Inject;
+import com.google.common.collect.Lists;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.List;
+
+public abstract class DataRequestHandler {
+  protected BeanJsonConverter converter;
+
+  @Inject
+  DataRequestHandler(BeanJsonConverter converter) {
+    this.converter = converter;
+  }
+
+  public void handleMethod(String httpMethod, HttpServletRequest servletRequest,
+      HttpServletResponse servletResponse, SecurityToken token)
+      throws IOException {
+    if (httpMethod == null || httpMethod.length() == 0) {
+      throw new IllegalArgumentException("Unserviced Http method type");
+    }
+    ResponseItem responseItem;
+    if (httpMethod.equals("GET")) {
+      responseItem = handleGet(servletRequest, token);
+    } else if (httpMethod.equals("POST")) {
+      responseItem = handlePost(servletRequest, token);
+    } else if (httpMethod.equals("PUT")) {
+      responseItem = handlePut(servletRequest, token);
+    } else if (httpMethod.equals("DELETE")) {
+      responseItem = handleDelete(servletRequest, token);
+    } else if (httpMethod.equals("HEAD")) {
+      responseItem = handleHead(servletRequest);
+    } else {
+      throw new IllegalArgumentException("Unserviced Http method type");
+    }
+    if (responseItem.getError() == null) {
+      PrintWriter writer = servletResponse.getWriter();
+      writer.write(converter.convertToJson(
+          responseItem.getResponse()).toString());
+    } else {
+      // throw an error
+    }
+  }
+
+  ResponseItem handleHead(HttpServletRequest servletRequest) {
+    throw new RuntimeException("Not Implemented");
+  }
+
+  abstract ResponseItem handleDelete(HttpServletRequest servletRequest,
+      SecurityToken token);
+
+  abstract ResponseItem handlePut(HttpServletRequest servletRequest,
+      SecurityToken token);
+
+  abstract ResponseItem handlePost(HttpServletRequest servletRequest,
+      SecurityToken token);
+
+  abstract ResponseItem handleGet(HttpServletRequest servletRequest,
+      SecurityToken token);
+
+  protected String[] getParamsFromRequest(HttpServletRequest servletRequest) {
+    return getQueryPath(servletRequest).split("/");
+  }
+
+  private String getQueryPath(HttpServletRequest servletRequest) {
+    String pathInfo = servletRequest.getPathInfo();
+    int index = pathInfo.indexOf('/', 1);
+    return pathInfo.substring(index + 1);
+  }
+
+  protected <T extends Enum<T>> T getEnumParam(
+      HttpServletRequest servletRequest, String paramName, T defaultValue,
+      Class<T> enumClass) {
+    String paramValue = servletRequest.getParameter(paramName);
+    if (paramValue != null) {
+      return Enum.valueOf(enumClass, paramValue);
+    }
+    return defaultValue;
+  }
+
+  protected int getIntegerParam(HttpServletRequest servletRequest,
+      String paramName, int defaultValue) {
+    String paramValue = servletRequest.getParameter(paramName);
+    if (paramValue != null) {
+      return new Integer(paramValue);
+    }
+    return defaultValue;
+  }
+
+  protected List<String> getListParam(HttpServletRequest servletRequest,
+      String paramName, List<String> defaultValue) {
+    String paramValue = servletRequest.getParameter(paramName);
+    if (paramValue != null) {
+      return Lists.newArrayList(paramValue.split(","));
+    }
+    return defaultValue;
+  }
+}

Added: incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/DataServiceServlet.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/DataServiceServlet.java?rev=666940&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/DataServiceServlet.java (added)
+++ incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/DataServiceServlet.java Wed Jun 11 18:07:28 2008
@@ -0,0 +1,136 @@
+/*
+ * 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.dataservice;
+
+import com.google.inject.Inject;
+import org.apache.shindig.common.SecurityTokenDecoder;
+import org.apache.shindig.common.SecurityToken;
+import org.apache.shindig.common.SecurityTokenException;
+import org.apache.shindig.common.servlet.InjectedServlet;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.util.Map;
+import java.util.logging.Logger;
+
+public class DataServiceServlet extends InjectedServlet {
+
+  public static enum GroupId {
+    ALL("@all"),
+    FRIENDS("@friends"),
+    SELF("@self"),
+    GROUP("not supported yet");
+
+    private final String jsonString;
+
+    GroupId(String jsonString) {
+      this.jsonString = jsonString;
+    }
+
+    public static GroupId fromJson(String s) {
+      return valueOf(s.substring(1).toUpperCase());
+    }
+  }
+
+  protected static final String X_HTTP_METHOD_OVERRIDE
+      = "X-HTTP-Method-Override";
+  protected static final String SECURITY_TOKEN_PARAM = "st";
+  protected static final String REQUEST_PARAMETER = "request";
+
+  public static final String PEOPLE_ROUTE = "people";
+  public static final String ACTIVITY_ROUTE = "activities";
+  public static final String APPDATA_ROUTE = "appdata";
+
+  private static final Logger logger = Logger.getLogger(
+      "org.apache.shindig.social.dataservice");
+
+  private SecurityTokenDecoder securityTokenDecoder;
+  private Map<String, DataRequestHandler> handlers;
+
+  @Inject
+  public void setHandlers(Map<String, DataRequestHandler> handlers) {
+    this.handlers = handlers;
+  }
+
+  @Inject
+  public void setSecurityTokenDecoder(SecurityTokenDecoder
+      securityTokenDecoder) {
+    this.securityTokenDecoder = securityTokenDecoder;
+  }
+
+  protected void doGet(HttpServletRequest servletRequest,
+      HttpServletResponse servletResponse)
+      throws ServletException, IOException {
+    doPost(servletRequest, servletResponse);
+  }
+
+  protected void doPut(HttpServletRequest servletRequest,
+      HttpServletResponse servletResponse)
+      throws ServletException, IOException {
+    doPost(servletRequest, servletResponse);
+  }
+
+  protected void doDelete(HttpServletRequest servletRequest,
+      HttpServletResponse servletResponse)
+      throws ServletException, IOException {
+    doPost(servletRequest, servletResponse);
+  }
+
+  protected void doPost(HttpServletRequest servletRequest,
+      HttpServletResponse servletResponse)
+      throws ServletException, IOException {
+    servletRequest.setCharacterEncoding("UTF-8");
+
+    String route = getRouteFromParameter(servletRequest.getPathInfo());
+    DataRequestHandler handler = handlers.get(route);
+    if (handler == null) {
+      throw new RuntimeException("No handler for route: "+route);
+    }
+
+    String method = getHttpMethodFromParameter(servletRequest.getMethod(),
+        servletRequest.getParameter(X_HTTP_METHOD_OVERRIDE));
+    try {
+      SecurityToken token = securityTokenDecoder.createToken(
+          servletRequest.getParameter(SECURITY_TOKEN_PARAM));
+      handler.handleMethod(method, servletRequest, servletResponse, token);
+    } catch (SecurityTokenException e) {
+      throw new RuntimeException(
+          "Implement error return for bad security token.");
+    }
+
+  }
+
+  /*package-protected*/ String getHttpMethodFromParameter(String method,
+      String overrideParameter) {
+    if (overrideParameter != null && overrideParameter.length() != 0) {
+      return overrideParameter;
+    } else {
+      return method;
+    }
+  }
+
+  /*package-protected*/ String getRouteFromParameter(String pathInfo) {
+    pathInfo = pathInfo.substring(1);
+    int indexOfNextPathSeparator = pathInfo.indexOf("/");
+    return indexOfNextPathSeparator != -1 ?
+        pathInfo.substring(0, indexOfNextPathSeparator) :
+        pathInfo;
+  }
+}

Added: incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/PersonHandler.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/PersonHandler.java?rev=666940&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/PersonHandler.java (added)
+++ incubator/shindig/trunk/java/social-api/src/main/java/org/apache/shindig/social/dataservice/PersonHandler.java Wed Jun 11 18:07:28 2008
@@ -0,0 +1,104 @@
+/*
+ * 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.dataservice;
+
+import org.apache.shindig.common.SecurityToken;
+import org.apache.shindig.social.ResponseError;
+import org.apache.shindig.social.ResponseItem;
+import org.apache.shindig.social.opensocial.model.Person;
+import org.apache.shindig.social.opensocial.util.BeanJsonConverter;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.inject.Inject;
+
+import javax.servlet.http.HttpServletRequest;
+import java.util.Set;
+
+
+public class PersonHandler extends DataRequestHandler {
+  private PersonService personService;
+
+  @Inject
+  public PersonHandler(PersonService personService,
+      BeanJsonConverter converter) {
+    super(converter);
+    this.personService = personService;
+  }
+
+  ResponseItem handleDelete(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    return new ResponseItem<Object>(ResponseError.BAD_REQUEST,
+        "You can't delete people. ", null);
+  }
+
+  ResponseItem handlePut(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    return new ResponseItem<Object>(ResponseError.NOT_IMPLEMENTED,
+        "You can't add people right now. ", null);  }
+
+  ResponseItem handlePost(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    return new ResponseItem<Object>(ResponseError.NOT_IMPLEMENTED,
+        "You can't add people right now. ", null);
+  }
+
+  /**
+   * /people/{userId}/{groupId}/{optionalPersonId}
+   *
+   * examples:
+   * /people/john.doe/@all
+   * /people/john.doe/@friends
+   * /people/john.doe/@self
+   */
+  ResponseItem handleGet(HttpServletRequest servletRequest,
+      SecurityToken token) {
+    String[] segments = getParamsFromRequest(servletRequest);
+
+    String userId = segments[0];
+    DataServiceServlet.GroupId groupId
+        = DataServiceServlet.GroupId.fromJson(segments[1]);
+
+    String optionalPersonId = null;
+    if (segments.length > 2) {
+      optionalPersonId = segments[2];
+    }
+
+    PersonService.SortOrder sort = getEnumParam(servletRequest, "orderBy",
+        PersonService.SortOrder.topFriends, PersonService.SortOrder.class);
+    PersonService.FilterType filter = getEnumParam(servletRequest, "filterBy",
+        PersonService.FilterType.all, PersonService.FilterType.class);
+
+    int first = getIntegerParam(servletRequest, "startIndex", 0);
+    int max = getIntegerParam(servletRequest, "count", 20);
+
+    Set<String>  profileDetails = Sets.newHashSet(
+        getListParam(servletRequest, "fields",
+            Lists.newArrayList(Person.Field.ID.toString(),
+                Person.Field.NAME.toString(),
+                Person.Field.THUMBNAIL_URL.toString())));
+
+    if (optionalPersonId != null
+        || groupId == DataServiceServlet.GroupId.SELF) {
+      return personService.getPerson(userId, token);
+    }
+    return personService.getPeople(userId, groupId, sort, filter, first, max,
+        profileDetails, token);
+  }
+
+}

Added: incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/dataservice/DataRequestHandlerTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/dataservice/DataRequestHandlerTest.java?rev=666940&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/dataservice/DataRequestHandlerTest.java (added)
+++ incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/dataservice/DataRequestHandlerTest.java Wed Jun 11 18:07:28 2008
@@ -0,0 +1,60 @@
+/*
+ * 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.dataservice;
+
+import junit.framework.TestCase;
+import org.easymock.classextension.EasyMock;
+import org.apache.shindig.social.ResponseItem;
+import org.apache.shindig.common.SecurityToken;
+
+import javax.servlet.http.HttpServletRequest;
+
+public class DataRequestHandlerTest extends TestCase {
+
+  public void testgetParamsFromRequest() throws Exception {
+    HttpServletRequest req = EasyMock.createMock(HttpServletRequest.class);
+    EasyMock.expect(req.getPathInfo()).andReturn("/people/5/@self");
+    DataRequestHandler drh = new DataRequestHandler(null) {
+      ResponseItem handleDelete(HttpServletRequest servletRequest,
+          SecurityToken token) {
+        return null;
+      }
+
+      ResponseItem handlePut(HttpServletRequest servletRequest,
+          SecurityToken token) {
+        return null;
+      }
+
+      ResponseItem handlePost(HttpServletRequest servletRequest,
+          SecurityToken token) {
+        return null;
+      }
+
+      ResponseItem handleGet(HttpServletRequest servletRequest,
+          SecurityToken token) {
+        return null;
+      }
+    };
+
+    EasyMock.replay(req);
+    String[] params = drh.getParamsFromRequest(req);
+    assertEquals("5", params[0]);
+    assertEquals("@self", params[1]);
+    EasyMock.verify(req);
+  }
+}
\ No newline at end of file

Added: incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/dataservice/DataServiceServletTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/dataservice/DataServiceServletTest.java?rev=666940&view=auto
==============================================================================
--- incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/dataservice/DataServiceServletTest.java (added)
+++ incubator/shindig/trunk/java/social-api/src/test/java/org/apache/shindig/social/dataservice/DataServiceServletTest.java Wed Jun 11 18:07:28 2008
@@ -0,0 +1,100 @@
+/*
+ * 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.dataservice;
+
+import org.apache.shindig.common.BasicSecurityTokenDecoder;
+import org.apache.shindig.common.SecurityTokenException;
+
+import com.google.common.collect.Maps;
+import junit.framework.TestCase;
+import org.easymock.classextension.EasyMock;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.util.HashMap;
+
+public class DataServiceServletTest extends TestCase {
+  private HttpServletRequest req;
+  private HttpServletResponse res;
+  private DataServiceServlet servlet ;
+  private PersonHandler peopleHandler;
+  private ActivityHandler activityHandler;
+  private AppDataHandler appDataHandler;
+
+  protected void setUp() throws Exception {
+    req = EasyMock.createMock(HttpServletRequest.class);
+    res = EasyMock.createMock(HttpServletResponse.class);
+
+    HashMap<String, DataRequestHandler> handlers = Maps.newHashMap();
+    peopleHandler = EasyMock.createMock(PersonHandler.class);
+    activityHandler = EasyMock.createMock(ActivityHandler.class);
+    appDataHandler = EasyMock.createMock(AppDataHandler.class);
+
+    handlers.put(DataServiceServlet.PEOPLE_ROUTE, peopleHandler);
+    handlers.put(DataServiceServlet.ACTIVITY_ROUTE, activityHandler);
+    handlers.put(DataServiceServlet.APPDATA_ROUTE, appDataHandler);
+    servlet = new DataServiceServlet();
+    servlet.setHandlers(handlers);
+    BasicSecurityTokenDecoder tokenDecoder
+        = EasyMock.createMock(BasicSecurityTokenDecoder.class);
+
+    servlet.setSecurityTokenDecoder(tokenDecoder);
+
+    EasyMock.expect(req.getMethod()).andReturn("POST");
+    req.setCharacterEncoding("UTF-8");
+  }
+
+
+  public void testPeopleUriRecognition() throws Exception {
+    verifyHandlerWasFoundForPathInfo("/"
+        + DataServiceServlet.PEOPLE_ROUTE+"/5/@self", peopleHandler);
+  }
+
+  public void testActivitiesUriRecognition() throws Exception {
+    verifyHandlerWasFoundForPathInfo("/"
+        + DataServiceServlet.ACTIVITY_ROUTE +"/5/@self", activityHandler);
+  }
+
+  public void testAppDataUriRecognition() throws Exception {
+    verifyHandlerWasFoundForPathInfo("/"
+        + DataServiceServlet.APPDATA_ROUTE+"/5/@self", appDataHandler);
+  }
+
+  public void testMethodOverride() throws Exception {
+    EasyMock.expect(req.getHeader("X-HTTP-Method-Override")).andReturn("GET");
+  }
+
+  private void verifyHandlerWasFoundForPathInfo(String peoplePathInfo,
+      DataRequestHandler handler) throws ServletException, IOException,
+      SecurityTokenException {
+    EasyMock.expect(req.getPathInfo()).andReturn(peoplePathInfo);
+    EasyMock.expect(req.getMethod()).andReturn("POST");
+    EasyMock.expect(req.getParameter(
+        DataServiceServlet.X_HTTP_METHOD_OVERRIDE)).andReturn("POST");
+    String tokenStr = "owner:viewer:app:container.com:foo:bar";
+    EasyMock.expect(req.getParameter(
+        DataServiceServlet.SECURITY_TOKEN_PARAM)).andReturn(tokenStr);
+    handler.handleMethod("POST", req, res, null);
+    EasyMock.replay(req, res, handler);
+    servlet.service(req, res);
+    EasyMock.verify(req, res, handler);
+  }
+
+}