You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by ra...@apache.org on 2015/10/09 06:17:24 UTC

[19/50] [abbrv] lens git commit: LENS-742 : Adds query feature : Saved query and parameterization

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryDao.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryDao.java b/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryDao.java
new file mode 100644
index 0000000..9dfde0d
--- /dev/null
+++ b/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryDao.java
@@ -0,0 +1,483 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.lens.server.query.save;
+
+import java.io.IOException;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+import javax.ws.rs.core.MultivaluedMap;
+
+import org.apache.lens.api.query.save.ListResponse;
+import org.apache.lens.api.query.save.Parameter;
+import org.apache.lens.api.query.save.SavedQuery;
+import org.apache.lens.server.api.error.LensException;
+import org.apache.lens.server.api.query.save.exception.SavedQueryNotFound;
+
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.ResultSetHandler;
+import org.apache.commons.lang3.StringEscapeUtils;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+public class SavedQueryDao {
+
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+  private static final String VALUE_ALIAS = "value_alias";
+  private static final String SAVED_QUERY_TABLE_NAME = "saved_query";
+  private static final String ID_COL_NAME = "id";
+  private static final String NAME_COL_NAME = "name";
+  private static final String DESCRIPTION_COL_NAME = "description";
+  private static final String QUERY_COL_NAME = "query";
+  private static final String PARAMS_COL_NAME = "params_json";
+  private static final String CREATED_AT_COL_NAME = "created_at";
+  private static final String UPDATED_AT_COL_NAME = "updated_at";
+
+  private final QueryRunner runner;
+  private final Dialect dialect;
+
+  SavedQueryDao(String dialectClass, QueryRunner runner)
+    throws LensException {
+    try {
+      this.runner = runner;
+      this.dialect = (Dialect) Class.forName(dialectClass).newInstance();
+      createSavedQueryTableIfNotExists();
+    } catch (Exception e) {
+      throw new LensException("Error initializing saved query dao", e);
+    }
+  }
+
+  /**
+   * Creates the saved query table
+   *
+   * @throws LensException
+   */
+  public void createSavedQueryTableIfNotExists() throws LensException {
+    try {
+      runner.update(dialect.getCreateTableSyntax());
+    } catch (SQLException e) {
+      throw new LensException("Cannot create saved query table!", e);
+    }
+  }
+
+  /**
+   * Saves the query passed
+   *
+   * @param savedQuery
+   * @return
+   * @throws LensException
+   */
+  public long saveQuery(SavedQuery savedQuery) throws LensException {
+    try {
+      final ECMAEscapedSavedQuery ecmaEscaped = ECMAEscapedSavedQuery.getFor(savedQuery);
+      runner.update(
+        "insert into " + SAVED_QUERY_TABLE_NAME + " values (" + dialect.getAutoIncrementId(runner) + ", "
+          + "'" + ecmaEscaped.getName() + "'"
+          + ", "
+          + "'" + ecmaEscaped.getDescription() + "'"
+          + ","
+          + "'" + ecmaEscaped.getQuery() + "'"
+          + ","
+          + "'" + ecmaEscaped.getParameters() + "'"
+          + ","
+          + "now()"
+          + ","
+          + "now()"
+          + ")"
+      );
+      return dialect.getLastInsertedID(runner);
+    } catch (SQLException e) {
+      throw new LensException("Save query failed !", e);
+    }
+  }
+
+  /**
+   * Updates the saved query id with new payload
+   *
+   * @param id
+   * @param savedQuery
+   * @throws LensException
+   */
+  public void updateQuery(long id, SavedQuery savedQuery) throws LensException {
+    try {
+      final ECMAEscapedSavedQuery ecmaEscaped = ECMAEscapedSavedQuery.getFor(savedQuery);
+      final int rowsUpdated = runner.update(
+        "update " + SAVED_QUERY_TABLE_NAME +"  set "
+          + NAME_COL_NAME + " = '" + ecmaEscaped.getName() + "',"
+          + DESCRIPTION_COL_NAME + " = '" + ecmaEscaped.getDescription() + "',"
+          + QUERY_COL_NAME + " = '" + ecmaEscaped.getQuery() + "',"
+          + PARAMS_COL_NAME + " = '" + ecmaEscaped.getParameters() + "',"
+          + UPDATED_AT_COL_NAME + " = now() "
+          + "where " + ID_COL_NAME + " = " + id
+      );
+      if (rowsUpdated == 0) {
+        throw new SavedQueryNotFound(id);
+      }
+    } catch (SQLException e) {
+      throw new LensException("Update failed for " + id, e);
+    }
+  }
+
+  /**
+   * Gets saved query with the given id
+   *
+   * @param id
+   * @return
+   * @throws LensException
+   */
+  public SavedQuery getSavedQueryByID(long id) throws LensException {
+    final List<SavedQuery> savedQueries;
+    try {
+      savedQueries = runner.query(
+        "select * from " + SAVED_QUERY_TABLE_NAME + " where " + ID_COL_NAME + " = " + id,
+        new SavedQueryResultSetHandler()
+      );
+    } catch (SQLException e) {
+      throw new LensException("Get failed for " + id, e);
+    }
+    int size = savedQueries.size();
+    switch (size) {
+    case 0:
+      throw new SavedQueryNotFound(id);
+    case 1:
+      return savedQueries.get(0);
+    default:
+      throw new RuntimeException("More than one obtained for id, Please check the integrity of the data!");
+    }
+  }
+
+  /**
+   * Returns a list of saved queries
+   *
+   * @param criteria  a multivalued map that has the filter criteria
+   * @param start     Displacement from the start of the search result
+   * @param count     Count of number of records required
+   * @return list of saved queries
+   * @throws LensException
+   */
+  public ListResponse getList(
+    MultivaluedMap<String, String> criteria, long start, long count) throws LensException {
+    final StringBuilder selectQueryBuilder = new StringBuilder("select * from " + SAVED_QUERY_TABLE_NAME);
+    final Set<String> availableFilterKeys = FILTER_KEYS.keySet();
+    final Sets.SetView<String> intersection = Sets.intersection(availableFilterKeys, criteria.keySet());
+    if (intersection.size() > 0) {
+      final StringBuilder whereClauseBuilder = new StringBuilder(" where ");
+      final List<String> predicates = Lists.newArrayList();
+      for (String colName : intersection) {
+        predicates.add(
+          FILTER_KEYS.get(colName)
+            .resolveFilterExpression(
+              colName,
+              criteria.getFirst(colName)
+            )
+        );
+      }
+      Joiner.on(" and ").skipNulls().appendTo(whereClauseBuilder, predicates);
+      selectQueryBuilder.append(whereClauseBuilder.toString());
+    }
+    final String listCountQuery = "select count(*) as " + VALUE_ALIAS
+      + " from (" + selectQueryBuilder.toString() + ") tmp_table";
+    selectQueryBuilder
+      .append(" limit ")
+      .append(start)
+      .append(", ")
+      .append(count);
+    final String listQuery = selectQueryBuilder.toString();
+    try {
+      return new ListResponse(
+        start,
+        runner.query(listCountQuery, new SingleValuedResultHandler()),
+        runner.query(listQuery, new SavedQueryResultSetHandler())
+      );
+    } catch (SQLException e) {
+      throw new LensException("List query failed!", e);
+    }
+  }
+
+  /**
+   * Deletes the saved query with the given id
+   *
+   * @param id
+   * @throws LensException
+   */
+  public void deleteSavedQueryByID(long id) throws LensException {
+    try {
+      int rowsDeleted = runner.update(
+        "delete from " + SAVED_QUERY_TABLE_NAME +" where " + ID_COL_NAME + " = " + id
+      );
+      if (rowsDeleted == 0) {
+        throw new SavedQueryNotFound(id);
+      } else if (rowsDeleted > 1) {
+        throw new LensException("Warning! More than one record was deleted", new Throwable());
+      }
+    } catch (SQLException e) {
+      throw new LensException("Delete query failed", e);
+    }
+  }
+
+  /**
+   * The interface Dialect.
+   */
+  public interface Dialect {
+    /**
+     * The create table syntax for 'this' dialect
+     * @return
+     */
+    String getCreateTableSyntax();
+
+    /**
+     * Method to get the auto increment id/keyword(null) for the ID column
+     * @param runner
+     * @return
+     * @throws SQLException
+     */
+    Long getAutoIncrementId(QueryRunner runner)  throws SQLException;
+
+    /**
+     * Get the last increment id after doing an auto increment
+     * @param runner
+     * @return
+     * @throws SQLException
+     */
+    Long getLastInsertedID(QueryRunner runner) throws SQLException;
+  }
+
+  /**
+   * MySQL dialect for saved query.
+   */
+  public static class MySQLDialect implements Dialect {
+
+    @Override
+    public String getCreateTableSyntax() {
+      return "CREATE TABLE IF NOT EXISTS " + SAVED_QUERY_TABLE_NAME + " ("
+        + ID_COL_NAME + " int(11) NOT NULL AUTO_INCREMENT,"
+        + NAME_COL_NAME + " varchar(255) NOT NULL,"
+        + DESCRIPTION_COL_NAME + " varchar(255) DEFAULT NULL,"
+        + QUERY_COL_NAME + " longtext,"
+        + PARAMS_COL_NAME + " longtext,"
+        + CREATED_AT_COL_NAME + " timestamp,"
+        + UPDATED_AT_COL_NAME + " timestamp,"
+        + "  PRIMARY KEY ("+ ID_COL_NAME +")"
+        + ")";
+
+    }
+
+    @Override
+    public Long getAutoIncrementId(QueryRunner runner) throws SQLException {
+      return null;
+    }
+
+    @Override
+    public Long getLastInsertedID(QueryRunner runner) throws SQLException {
+      return runner.query(
+        "select last_insert_id() as " + VALUE_ALIAS,
+        new SingleValuedResultHandler()
+      );
+    }
+  }
+
+  /**
+   * HSQL dialect for saved query (Used with testing).
+   */
+  public static class HSQLDialect implements Dialect {
+
+    @Override
+    public String getCreateTableSyntax() {
+      return "CREATE TABLE if not exists " + SAVED_QUERY_TABLE_NAME + "  ("
+        + ID_COL_NAME + " int GENERATED BY DEFAULT AS IDENTITY PRIMARY KEY, "
+        + NAME_COL_NAME + " varchar(255), "
+        + DESCRIPTION_COL_NAME + " varchar(255), "
+        + QUERY_COL_NAME + " varchar(255), "
+        + PARAMS_COL_NAME + " varchar(255), "
+        + CREATED_AT_COL_NAME + " timestamp, "
+        + UPDATED_AT_COL_NAME + " timestamp)";
+
+    }
+
+    @Override
+    public Long getAutoIncrementId(QueryRunner runner) throws SQLException {
+      return runner.query("select max(" + ID_COL_NAME + ")  as " + VALUE_ALIAS +" from " + SAVED_QUERY_TABLE_NAME
+        , new SingleValuedResultHandler()) + 1;
+    }
+
+    @Override
+    public Long getLastInsertedID(QueryRunner runner) throws SQLException {
+      Long id = runner.query("select max(" + ID_COL_NAME + ")  as " + VALUE_ALIAS + " from " + SAVED_QUERY_TABLE_NAME
+        , new SingleValuedResultHandler());
+      if (id == 0) {
+        id++;
+      }
+      return id;
+    }
+  }
+
+  /**
+   * Result set handler class to get a saved query from result set
+   */
+  public static class SavedQueryResultSetHandler implements ResultSetHandler<List<SavedQuery>> {
+
+    @Override
+    public List<SavedQuery> handle(ResultSet resultSet) throws SQLException {
+      List<SavedQuery> queries = Lists.newArrayList();
+      while (resultSet.next()) {
+        long id = resultSet.getLong(ID_COL_NAME);
+        final String name = StringEscapeUtils.unescapeEcmaScript(resultSet.getString(NAME_COL_NAME));
+        final String description = StringEscapeUtils.unescapeEcmaScript(resultSet.getString(DESCRIPTION_COL_NAME));
+        final String query = StringEscapeUtils.unescapeEcmaScript(resultSet.getString(QUERY_COL_NAME));
+        final List<Parameter> parameterList;
+        try {
+          parameterList = deserializeFrom(
+            StringEscapeUtils.unescapeEcmaScript(resultSet.getString(PARAMS_COL_NAME))
+          );
+        } catch (LensException e) {
+          throw new SQLException("Cannot deserialize parameters ", e);
+        }
+        queries.add(new SavedQuery(
+          id,
+          name,
+          description,
+          query,
+          parameterList
+        ));
+      }
+      return queries;
+    }
+  }
+
+  /**
+   * Result set handler class to get a the last inserted ID from the resultset
+   */
+  public static class SingleValuedResultHandler implements ResultSetHandler<Long> {
+
+    @Override
+    public Long handle(ResultSet resultSet) throws SQLException {
+      while (resultSet.next()) {
+        return resultSet.getLong(VALUE_ALIAS);
+      }
+      throw new SQLException("For cursor : " + resultSet.getCursorName());
+    }
+  }
+
+  @AllArgsConstructor
+  @Data
+  /**
+   * This class represents a ECMA escaped version of saved query,
+   * that can be safely inserted into DB
+   */
+  private static class ECMAEscapedSavedQuery {
+    private final long id;
+    private final String name;
+    private final String description;
+    private final String query;
+    private final String parameters;
+
+    static ECMAEscapedSavedQuery getFor(SavedQuery savedQuery) throws LensException {
+      return new ECMAEscapedSavedQuery(
+        savedQuery.getId(),
+        StringEscapeUtils.escapeEcmaScript(savedQuery.getName()),
+        StringEscapeUtils.escapeEcmaScript(savedQuery.getDescription()),
+        StringEscapeUtils.escapeEcmaScript(savedQuery.getQuery()),
+        StringEscapeUtils.escapeEcmaScript(serializeParameters(savedQuery))
+      );
+    }
+  }
+
+  /**
+   * The filter data type used in the list api
+   */
+  enum FilterDataType {
+    STRING {
+      String resolveFilterExpression(String col, String val) {
+        return " " + col + " like '%" + val + "%'";
+      }
+    },
+    NUMBER {
+      String resolveFilterExpression(String col, String val) {
+        return col + "=" + Long.parseLong(val);
+      }
+    },
+    BOOLEAN {
+      String resolveFilterExpression(String col, String val) {
+        return col + "=" + Boolean.parseBoolean(val);
+      }
+    };
+
+    abstract String resolveFilterExpression(String col, String val);
+  }
+
+  /**
+   * Map of available filter keys and their data types
+   * The list api can have filter criteria based on these keys.
+   */
+  private static final ImmutableMap<String, FilterDataType> FILTER_KEYS;
+
+  static {
+    final ImmutableMap.Builder<String, FilterDataType> filterValuesBuilder = ImmutableMap.builder();
+    filterValuesBuilder.put(NAME_COL_NAME, FilterDataType.STRING);
+    filterValuesBuilder.put(DESCRIPTION_COL_NAME, FilterDataType.STRING);
+    filterValuesBuilder.put(QUERY_COL_NAME, FilterDataType.STRING);
+    filterValuesBuilder.put(ID_COL_NAME, FilterDataType.NUMBER);
+    FILTER_KEYS = filterValuesBuilder.build();
+  }
+
+
+  /**
+   * Serializes the parameters of saved query using jackson
+   *
+   * @param savedQuery
+   * @return
+   * @throws LensException
+   */
+  private static String serializeParameters(SavedQuery savedQuery) throws LensException {
+    final String paramsJson;
+    try {
+      paramsJson = MAPPER.writeValueAsString(savedQuery.getParameters());
+    } catch (JsonProcessingException e) {
+      throw new LensException("Serialization failed for " + savedQuery.getParameters(), e);
+    }
+    return paramsJson;
+  }
+
+  /**
+   * Deserializes the parameters from string using jackson
+   *
+   * @param paramsJson
+   * @return
+   * @throws LensException
+   */
+  private static List<Parameter> deserializeFrom(String paramsJson) throws LensException {
+    final Parameter[] parameterArray;
+    try {
+      parameterArray = MAPPER.readValue(paramsJson, Parameter[].class);
+    } catch (IOException e) {
+      throw new LensException("Failed to deserialize from " + paramsJson, e);
+    }
+    return Arrays.asList(parameterArray);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryResource.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryResource.java b/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryResource.java
new file mode 100644
index 0000000..a7e0aef
--- /dev/null
+++ b/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryResource.java
@@ -0,0 +1,307 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.lens.server.query.save;
+
+import static org.apache.lens.api.error.LensCommonErrorCode.INVALID_XML_ERROR;
+import static org.apache.lens.api.query.save.ResourceModifiedResponse.Action.CREATED;
+import static org.apache.lens.api.query.save.ResourceModifiedResponse.Action.DELETED;
+import static org.apache.lens.api.query.save.ResourceModifiedResponse.Action.UPDATED;
+import static org.apache.lens.server.api.LensConfConstants.*;
+
+import java.io.IOException;
+
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.*;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.lens.api.LensConf;
+import org.apache.lens.api.LensSessionHandle;
+import org.apache.lens.api.error.ErrorCollection;
+import org.apache.lens.api.query.QuerySubmitResult;
+import org.apache.lens.api.query.save.ListResponse;
+import org.apache.lens.api.query.save.ParameterParserResponse;
+import org.apache.lens.api.query.save.ResourceModifiedResponse;
+import org.apache.lens.api.query.save.SavedQuery;
+import org.apache.lens.api.result.LensAPIResult;
+import org.apache.lens.cube.parse.HQLParser;
+import org.apache.lens.server.LensServices;
+import org.apache.lens.server.api.LensErrorInfo;
+import org.apache.lens.server.api.error.LensException;
+import org.apache.lens.server.api.query.QueryExecutionService;
+import org.apache.lens.server.api.query.save.*;
+import org.apache.lens.server.api.query.save.param.ParameterParser;
+import org.apache.lens.server.api.query.save.param.ParameterResolver;
+import org.apache.lens.server.model.LogSegregationContext;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+import org.glassfish.grizzly.http.server.Response;
+import org.glassfish.jersey.media.multipart.FormDataParam;
+
+import lombok.NonNull;
+
+@Path("/queryapi")
+/**
+ * Saved query resource
+ * <p></p>
+ * CRUD on saved query
+ */
+public class SavedQueryResource {
+
+  final SavedQueryService savedQueryService;
+  final QueryExecutionService queryService;
+  private final ErrorCollection errorCollection;
+  private final LogSegregationContext logSegregationContext;
+  private static final String DEFAULT_START = "0";
+  private final int defaultCount;
+
+  public SavedQueryResource() {
+    savedQueryService = LensServices.get().getService(SavedQueryServiceImpl.NAME);
+    queryService = LensServices.get().getService(QueryExecutionService.NAME);
+    errorCollection = LensServices.get().getErrorCollection();
+    logSegregationContext = LensServices.get().getLogSegregationContext();
+    final HiveConf hiveConf = LensServices.get().getHiveConf();
+    defaultCount = hiveConf.getInt(FETCH_COUNT_SAVED_QUERY_LIST_KEY, DEFAULT_FETCH_COUNT_SAVED_QUERY_LIST);
+  }
+
+  @GET
+  @Path("/savedqueries/health")
+  @Produces(MediaType.TEXT_PLAIN)
+  /**
+   * Health check end point
+   */
+  public String getMessage() {
+    return "Saved query api is up";
+  }
+
+
+  /**
+   * Gets a list of saved queries matching the criteria (url parameters)
+   * windowed by count and start.
+   *
+   * @param sessionid  The sessionid in which user is working
+   * @param info       URI context injected for query parameters
+   * @param start      Offset to start from the search result
+   * @param count      Number of records to fetch from start
+   * @return {@link org.apache.lens.api.query.save.ListResponse} ListResponse object
+   * @throws LensException
+   */
+  @GET
+  @Path("/savedqueries")
+  @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  public ListResponse getList(
+    @QueryParam("sessionid") LensSessionHandle sessionid,
+    @Context UriInfo info,
+    @DefaultValue(DEFAULT_START) @QueryParam("start") int start,
+    @QueryParam("count") String count) throws LensException {
+    try {
+      final int countVal = count == null? defaultCount: Integer.parseInt(count);
+      return savedQueryService.list(info.getQueryParameters(), start, countVal);
+    } catch (LensException e) {
+      throw getWrapped(e);
+    }
+  }
+
+  /**
+   * Gets the saved query with the given id.
+   *
+   * @param sessionid  The sessionid in which user is working
+   * @param id         id of the saved query
+   * @return {@link org.apache.lens.api.query.save.SavedQuery} SavedQuery object
+   * @throws LensException
+   */
+  @GET
+  @Path("/savedqueries/{id}")
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  public SavedQuery getByID(
+    @QueryParam("sessionid") LensSessionHandle sessionid,
+    @PathParam("id") long id) throws LensException {
+    try {
+      return savedQueryService.get(id);
+    } catch (LensException e) {
+      throw getWrapped(e);
+    }
+  }
+
+  /**
+   * Deletes the saved query with the given id.
+   *
+   * @param sessionid  The sessionid in which user is working
+   * @param id         id of the saved query
+   * @return {@link org.apache.lens.api.query.save.ResourceModifiedResponse} ResourceModifiedResponse object
+   * @throws LensException
+   */
+  @DELETE
+  @Path("/savedqueries/{id}")
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  public ResourceModifiedResponse deleteById(
+    @QueryParam("sessionid") LensSessionHandle sessionid,
+    @PathParam("id") long id) throws LensException {
+    try {
+      savedQueryService.delete(id);
+      return new ResourceModifiedResponse(id, "saved_query", DELETED);
+    } catch (LensException e) {
+      throw getWrapped(e);
+    }
+  }
+
+  /**
+   * Creates a new saved query.
+   *
+   * @param sessionid   The sessionid in which user is working
+   * @param savedQuery  Saved query object
+   * @param response    Injected response context object
+   * @return {@link org.apache.lens.api.query.save.ResourceModifiedResponse} ResourceModifiedResponse object
+   * @throws LensException
+   * @throws IOException
+   */
+  @POST
+  @Path(("/savedqueries"))
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  public ResourceModifiedResponse create(
+    @QueryParam("sessionid") LensSessionHandle sessionid,
+    SavedQuery savedQuery,
+    @Context final Response response)
+    throws LensException, IOException {
+    try {
+      validateSampleResolved(savedQuery);
+      long id = savedQueryService.save(savedQuery);
+      response.setStatus(HttpServletResponse.SC_CREATED);
+      response.flush();
+      return new ResourceModifiedResponse(id, "saved_query", CREATED);
+    } catch (LensException e) {
+      throw getWrapped(e);
+    }
+  }
+
+  /**
+   * Updates the saved query {id} with the new payload.
+   *
+   * @param sessionid   The sessionid in which user is working
+   * @param savedQuery  Saved query object
+   * @param response    Injected response context object
+   * @return {@link org.apache.lens.api.query.save.ResourceModifiedResponse} ResourceModifiedResponse object
+   * @throws LensException
+   * @throws IOException
+   */
+  @PUT
+  @Path("/savedqueries/{id}")
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  public ResourceModifiedResponse update(
+    @QueryParam("sessionid") LensSessionHandle sessionid,
+    @PathParam("id") long id,
+    SavedQuery savedQuery,
+    @Context final Response response) throws LensException, IOException {
+    try {
+      validateSampleResolved(savedQuery);
+      savedQueryService.update(id, savedQuery);
+      response.setStatus(HttpServletResponse.SC_CREATED);
+      response.flush();
+      return new ResourceModifiedResponse(id, "saved_query", UPDATED);
+    } catch (LensException e) {
+      throw getWrapped(e);
+    }
+  }
+
+  /**
+   * Parses the query and returns parameters that are found in the query.
+   *
+   * @param sessionid  The sessionid in which user is working
+   * @param query      The HQL query
+   * @return {@link org.apache.lens.api.query.save.ParameterParserResponse} ParameterParserResponse object
+   */
+  @GET
+  @Path("/savedqueries/parameters")
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  public ParameterParserResponse getParameters(
+    @QueryParam("sessionid") LensSessionHandle sessionid,
+    @QueryParam("query") String query) {
+    return new ParameterParser(query).extractParameters();
+  }
+
+  /**
+   * Runs the saved query with the given id and returns a query handle.
+   *
+   * @param id         id of the saved query
+   * @param info       Injected UriInfo context object
+   * @param sessionid  The sessionid in which user is working
+   * @param conf       Lens configuration overrides for the query
+   * @return LensAPIResult containing the query handle
+   * @throws LensException
+   */
+  @POST
+  @Path("/savedqueries/{id}")
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, MediaType.TEXT_PLAIN})
+  public LensAPIResult<? extends QuerySubmitResult> run(
+    @PathParam("id") long id,
+    @Context UriInfo info,
+    @FormDataParam("sessionid") LensSessionHandle sessionid,
+    @FormDataParam("conf") LensConf conf) throws LensException {
+    final String requestId = this.logSegregationContext.getLogSegragationId();
+    try {
+      final SavedQuery savedQuery = savedQueryService.get(id);
+      final String query = ParameterResolver.resolve(savedQuery, info.getQueryParameters());
+      return LensAPIResult.composedOf(
+        null,
+        requestId,
+        queryService.executeAsync(sessionid, query, conf, savedQuery.getName())
+      );
+    } catch (LensException e) {
+      throw getWrapped(e);
+    }
+  }
+
+  /**
+   * Helper method that builds error response for LensException provided.
+   *
+   * @param e    Lens exception object
+   * @return lens exception object with error response built
+   * @throws LensException
+   */
+  private LensException getWrapped(LensException e) throws
+    LensException {
+    e.buildLensErrorResponse(errorCollection, null, logSegregationContext.getLogSegragationId());
+    throw e;
+  }
+
+  /**
+   * Validates the saved query and throws LensException with.
+   * BAD_SYNTAX code if wrong
+   *
+   * @param savedQuery Saved query object
+   * @throws LensException if invalid
+   */
+  private void validateSampleResolved(@NonNull SavedQuery savedQuery) throws LensException {
+    final String sampleResolved  = SavedQueryHelper.getSampleResolvedQuery(savedQuery);
+    try {
+      HQLParser.parseHQL(sampleResolved, new HiveConf());
+    } catch (Exception e) {
+      throw new LensException(
+        new LensErrorInfo(INVALID_XML_ERROR.getValue(), 0, INVALID_XML_ERROR.toString())
+        , e
+        , "Encountered while resolving with sample values { " + sampleResolved + " }");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryServiceImpl.java
new file mode 100644
index 0000000..d9de897
--- /dev/null
+++ b/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryServiceImpl.java
@@ -0,0 +1,141 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.lens.server.query.save;
+
+import javax.ws.rs.core.MultivaluedMap;
+
+import org.apache.lens.api.query.save.ListResponse;
+import org.apache.lens.api.query.save.SavedQuery;
+import org.apache.lens.server.BaseLensService;
+import org.apache.lens.server.api.LensConfConstants;
+import org.apache.lens.server.api.error.LensException;
+import org.apache.lens.server.api.health.HealthStatus;
+import org.apache.lens.server.api.query.save.SavedQueryService;
+import org.apache.lens.server.util.UtilityMethods;
+
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hive.service.cli.CLIService;
+
+import lombok.NonNull;
+
+public class SavedQueryServiceImpl extends BaseLensService implements SavedQueryService {
+  private SavedQueryDao dao;
+  private Configuration conf;
+
+  public static final String NAME = "savedquery";
+
+  /**
+   * Instantiates a new lens service.
+   *
+   * @param cliService the cli service
+   */
+  public SavedQueryServiceImpl(CLIService cliService) throws LensException {
+    super(NAME, cliService);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public synchronized void init(HiveConf hiveConf) {
+    super.init(hiveConf);
+    conf = hiveConf;
+    @NonNull final String dialect = conf.get(LensConfConstants.JDBC_DIALECT_PROVIDER_CLASS_KEY
+      , SavedQueryDao.HSQLDialect.class.getCanonicalName());
+    try {
+      dao = new SavedQueryDao(
+        dialect,
+        new QueryRunner(UtilityMethods.getPoolingDataSourceFromConf(conf))
+      );
+    } catch (LensException e) {
+      throw new RuntimeException("Cannot initialize saved query service", e);
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public HealthStatus getHealthStatus() {
+    return this.getServiceState().equals(STATE.STARTED)
+      ? new HealthStatus(true, "Saved query service is healthy.")
+      : new HealthStatus(false, "Saved query service is down.");
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public long save(SavedQuery savedQuery) throws LensException {
+    return dao.saveQuery(savedQuery);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void update(long id, SavedQuery savedQuery) throws LensException {
+    dao.updateQuery(id, savedQuery);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void delete(long id) throws LensException {
+    dao.deleteSavedQueryByID(id);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public SavedQuery get(long id) throws LensException {
+    return dao.getSavedQueryByID(id);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public ListResponse list(
+    MultivaluedMap<String, String> criteria, long start, long count) throws LensException {
+    return dao.getList(criteria, start, count);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void grant(long id, String sharingUser, String targetUserPath, String[] privileges)
+    throws LensException {
+    //NOOP
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void revoke(long id, String sharingUser, String targetUserPath, String[] privileges)
+    throws LensException {
+    //NOOP
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server/src/main/java/org/apache/lens/server/util/UtilityMethods.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/util/UtilityMethods.java b/lens-server/src/main/java/org/apache/lens/server/util/UtilityMethods.java
index 5d77eb7..a6c7b13 100644
--- a/lens-server/src/main/java/org/apache/lens/server/util/UtilityMethods.java
+++ b/lens-server/src/main/java/org/apache/lens/server/util/UtilityMethods.java
@@ -29,9 +29,10 @@ import javax.sql.DataSource;
 
 import org.apache.lens.server.api.LensConfConstants;
 
-import org.apache.commons.dbcp.BasicDataSource;
+import org.apache.commons.dbcp.*;
 import org.apache.commons.dbutils.QueryRunner;
 import org.apache.commons.dbutils.ResultSetHandler;
+import org.apache.commons.pool.impl.GenericObjectPool;
 import org.apache.hadoop.conf.Configuration;
 
 
@@ -141,6 +142,22 @@ public final class UtilityMethods {
     return tmp;
   }
 
+  public static DataSource getPoolingDataSourceFromConf(Configuration conf) {
+    final ConnectionFactory cf = new DriverManagerConnectionFactory(
+      conf.get(LensConfConstants.SERVER_DB_JDBC_URL, LensConfConstants.DEFAULT_SERVER_DB_JDBC_URL),
+      conf.get(LensConfConstants.SERVER_DB_JDBC_USER, LensConfConstants.DEFAULT_SERVER_DB_USER),
+      conf.get(LensConfConstants.SERVER_DB_JDBC_PASS, LensConfConstants.DEFAULT_SERVER_DB_PASS));
+    final GenericObjectPool connectionPool = new GenericObjectPool();
+    connectionPool.setTestOnBorrow(false);
+    connectionPool.setTestOnReturn(false);
+    connectionPool.setTestWhileIdle(true);
+    new PoolableConnectionFactory(cf, connectionPool, null
+      , conf.get(LensConfConstants.SERVER_DB_VALIDATION_QUERY,
+        LensConfConstants.DEFAULT_SERVER_DB_VALIDATION_QUERY), false, false)
+      .setDefaultAutoCommit(true);
+    return new PoolingDataSource(connectionPool);
+  }
+
   /**
    * Conf to string.
    *

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server/src/main/resources/lensserver-default.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/main/resources/lensserver-default.xml b/lens-server/src/main/resources/lensserver-default.xml
index 5d33eda..6a1ac7f 100644
--- a/lens-server/src/main/resources/lensserver-default.xml
+++ b/lens-server/src/main/resources/lensserver-default.xml
@@ -55,7 +55,7 @@
 
   <property>
     <name>lens.server.servicenames</name>
-    <value>session,query,metastore,scheduler,quota</value>
+    <value>session,query,metastore,scheduler,quota,savedquery</value>
     <description>These services would be started in the specified order when lens-server starts up</description>
   </property>
 
@@ -72,6 +72,12 @@
   </property>
 
   <property>
+    <name>lens.server.savedquery.service.impl</name>
+    <value>org.apache.lens.server.query.save.SavedQueryServiceImpl</value>
+    <description>Implementation class for saved query service</description>
+  </property>
+
+  <property>
     <name>lens.server.quota.service.impl</name>
     <value>org.apache.lens.server.quota.QuotaServiceImpl</value>
     <description>Implementation class for quota service</description>
@@ -360,7 +366,7 @@
 
   <property>
     <name>lens.server.ws.resourcenames</name>
-    <value>session,metastore,query,quota,scheduler,index,log</value>
+    <value>session,metastore,query,quota,scheduler,index,log,savedquery</value>
     <description>These JAX-RS resources would be started in the specified order when lens-server starts up</description>
   </property>
 
@@ -371,6 +377,12 @@
   </property>
 
   <property>
+    <name>lens.server.savedquery.ws.resource.impl</name>
+    <value>org.apache.lens.server.query.save.SavedQueryResource</value>
+    <description>Implementation class for Saved query Resource</description>
+  </property>
+
+  <property>
     <name>lens.server.query.ws.resource.impl</name>
     <value>org.apache.lens.server.query.QueryServiceResource</value>
     <description>Implementation class for Query Resource</description>
@@ -753,4 +765,21 @@
       implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.</description>
   </property>
 
+  <property>
+    <name>lens.server.savedquery.jdbc.dialectclass</name>
+    <value>org.apache.lens.server.query.save.SavedQueryDao$HSQLDialect</value>
+    <description>Dialect of the target DB, Default is HSQL. Override with the target DB used.</description>
+  </property>
+
+  <property>
+    <name>lens.server.savedquery.list.default.offset</name>
+    <value>0</value>
+    <description>Key denoting the default start value of saved query list api.</description>
+  </property>
+  <property>
+    <name>lens.server.savedquery.list.default.count</name>
+    <value>20</value>
+    <description>Key denoting the default fetch value of saved query list api.</description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server/src/test/java/org/apache/lens/server/query/save/TestSavedQueryService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/save/TestSavedQueryService.java b/lens-server/src/test/java/org/apache/lens/server/query/save/TestSavedQueryService.java
new file mode 100644
index 0000000..7c6ce49
--- /dev/null
+++ b/lens-server/src/test/java/org/apache/lens/server/query/save/TestSavedQueryService.java
@@ -0,0 +1,274 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.lens.server.query.save;
+
+import static org.apache.lens.api.query.save.ParameterCollectionType.SINGLE;
+import static org.apache.lens.api.query.save.ParameterDataType.STRING;
+
+import static org.testng.Assert.*;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.*;
+
+import org.apache.lens.api.LensSessionHandle;
+import org.apache.lens.api.jaxb.LensJAXBContextResolver;
+import org.apache.lens.api.query.save.*;
+import org.apache.lens.server.LensJerseyTest;
+import org.apache.lens.server.LensServices;
+import org.apache.lens.server.api.metrics.MetricsService;
+import org.apache.lens.server.api.query.QueryExecutionService;
+import org.apache.lens.server.api.query.save.SavedQueryService;
+import org.apache.lens.server.error.LensExceptionMapper;
+import org.apache.lens.server.query.QueryExecutionServiceImpl;
+
+import org.glassfish.jersey.client.ClientConfig;
+import org.glassfish.jersey.media.multipart.MultiPartFeature;
+import org.testng.annotations.AfterTest;
+import org.testng.annotations.BeforeTest;
+import org.testng.annotations.Test;
+
+import com.beust.jcommander.internal.Maps;
+import com.beust.jcommander.internal.Sets;
+import com.google.common.collect.Lists;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+@Test(groups = "unit-test")
+public class TestSavedQueryService extends LensJerseyTest {
+  SavedQueryServiceImpl savedQueryService;
+  QueryExecutionServiceImpl queryService;
+  MetricsService metricsSvc;
+  LensSessionHandle lensSessionId;
+
+  private static final String QUERY_STRING = "select * from table where "
+    + "col = :param1 ";
+
+  private static final SavedQuery QUERY = new SavedQuery(
+    1,
+    "query_name",
+    "description",
+    QUERY_STRING
+    ,
+    Lists.newArrayList(
+      new Parameter(
+        "param1", "Param1", new String[]{"val"}, STRING, SINGLE
+      )
+    )
+  );
+
+  public static class SavedQueryTestApp extends SavedQueryApp {
+
+    @Override
+    public Set<Class<?>> getClasses() {
+      final Set<Class<?>> classes = super.getClasses();
+      classes.add(LensExceptionMapper.class);
+      classes.add(LensJAXBContextResolver.class);
+      return classes;
+    }
+  }
+
+  @BeforeTest
+  public void setUp() throws Exception {
+    super.setUp();
+    savedQueryService = LensServices.get().getService(SavedQueryService.NAME);
+    queryService = LensServices.get().getService(QueryExecutionService.NAME);
+    metricsSvc = LensServices.get().getService(MetricsService.NAME);
+    Map<String, String> sessionconf = Maps.newHashMap();
+    sessionconf.put("test.session.key", "svalue");
+    lensSessionId = queryService.openSession("foo", "bar", sessionconf); // @localhost should be removed
+  }
+
+  @AfterTest
+  public void tearDown() throws Exception {
+    super.tearDown();
+    queryService.closeSession(lensSessionId);
+    super.tearDown();
+  }
+
+  @Override
+  protected Application configure() {
+    return new SavedQueryTestApp();
+  }
+
+  @Override
+  protected void configureClient(ClientConfig config) {
+    config.register(MultiPartFeature.class);
+    config.register(LensJAXBContextResolver.class);
+  }
+
+  @Test
+  public void testResource() throws InterruptedException {
+    assertEquals(
+      savedQueriesRoot().path("health").request().get().getStatus(),
+      200,
+      "Saved query resource is not up"
+    );
+  }
+
+  private WebTarget savedQueriesRoot() {
+    return target()
+      .path("queryapi")
+      .path("savedqueries");
+  }
+
+  private ResourceModifiedResponse updateQuery(long id) {
+    Response savedquery = savedQueriesRoot()
+      .path(String.valueOf(id))
+      .request(MediaType.APPLICATION_JSON_TYPE)
+      .accept(MediaType.APPLICATION_JSON_TYPE)
+      .put(Entity.json(QUERY));
+    savedquery.getStringHeaders().putSingle(HttpHeaders.CONTENT_TYPE, "application/json");
+    return savedquery.readEntity(ResourceModifiedResponse.class);
+  }
+
+  private ResourceModifiedResponse deleteQuery(long id) {
+    Response savedquery = savedQueriesRoot()
+      .path(String.valueOf(id))
+      .request(MediaType.APPLICATION_JSON_TYPE)
+      .accept(MediaType.APPLICATION_JSON_TYPE)
+      .delete();
+    savedquery.getStringHeaders().putSingle(HttpHeaders.CONTENT_TYPE, "application/json");
+    return savedquery.readEntity(ResourceModifiedResponse.class);
+  }
+
+  private SavedQuery get(long id) {
+    Response savedquery = savedQueriesRoot()
+      .path(String.valueOf(id))
+      .request(MediaType.APPLICATION_JSON_TYPE)
+      .accept(MediaType.APPLICATION_JSON_TYPE)
+      .get();
+    savedquery.getStringHeaders().putSingle(HttpHeaders.CONTENT_TYPE, "application/json");
+    return savedquery.readEntity(SavedQuery.class);
+  }
+
+  private ParameterParserResponse extractParameters() {
+    Response parameters = savedQueriesRoot()
+      .path("parameters")
+      .queryParam("query", QUERY_STRING)
+      .request(MediaType.APPLICATION_JSON_TYPE)
+      .accept(MediaType.APPLICATION_JSON_TYPE)
+      .get();
+    parameters.getStringHeaders().putSingle(HttpHeaders.CONTENT_TYPE, "application/json");
+    return parameters.readEntity(ParameterParserResponse.class);
+  }
+
+  private ResourceModifiedResponse saveQuery() {
+    Response savedquery = savedQueriesRoot()
+      .request(MediaType.APPLICATION_JSON_TYPE)
+      .accept(MediaType.APPLICATION_JSON_TYPE)
+      .post(Entity.json(QUERY));
+    savedquery.getStringHeaders().putSingle(HttpHeaders.CONTENT_TYPE, "application/json");
+    return savedquery.readEntity(ResourceModifiedResponse.class);
+  }
+
+  private ListResponse list(long offset, long count) {
+    Response savedquery = savedQueriesRoot()
+      .queryParam("start", offset)
+      .queryParam("count", count)
+      .request(MediaType.APPLICATION_JSON_TYPE)
+      .accept(MediaType.APPLICATION_JSON_TYPE)
+      .get();
+    savedquery.getStringHeaders().putSingle(HttpHeaders.CONTENT_TYPE, "application/json");
+    return savedquery.readEntity(ListResponse.class);
+  }
+
+  @Test
+  public void testSaveQuery() {
+    assertEquals(saveQuery().getStatus(), ResourceModifiedResponse.Action.CREATED);
+  }
+
+  @Test
+  public void testUpdateQuery() {
+    ResourceModifiedResponse saved = saveQuery();
+    ResourceModifiedResponse updated = updateQuery(saved.getId());
+    assertEquals(updated.getStatus(), ResourceModifiedResponse.Action.UPDATED);
+    assertEquals(updated.getId(), saved.getId());
+  }
+
+  @Test
+  public void testUpdateQueryNonExistentResource() {
+    try {
+      updateQuery(99999);
+      fail("Did not fail when querying for a non existent resource");
+    } catch (Throwable e) {
+      assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testGetQuery() {
+    final long id = saveQuery().getId();
+    SavedQuery savedQuery = get(id);
+    assertEquals(savedQuery.getId(), id);
+  }
+
+  @Test
+  public void testGetQueryNonExistentResource() {
+    try {
+      get(99999);
+      fail("Did not fail when querying for a non existent resource");
+    } catch (Throwable e) {
+      assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testListQuery() {
+    final Set<Long> ids = Sets.newHashSet();
+    ids.add(saveQuery().getId());
+    ids.add(saveQuery().getId());
+    ids.add(saveQuery().getId());
+    ids.add(saveQuery().getId());
+    final ListResponse list = list(0, 4);
+    final List<SavedQuery> queries = list.getResoures();
+    assertEquals(ids.size(), queries.size());
+  }
+
+  @Test
+  public void testDeleteQuery() {
+    long id = saveQuery().getId();
+    deleteQuery(id);
+    try {
+      get(id);
+      fail("Resource not deleted");
+    } catch (Throwable e) {
+      assertTrue(true);
+    }
+
+  }
+
+  @Test
+  public void testDeleteQueryNonExistentResource() {
+    long id = saveQuery().getId();
+    deleteQuery(id);
+    try {
+      deleteQuery(id);
+      fail("Succeeded in deleting a non existent resource");
+    } catch (Throwable e) {
+      assertTrue(true);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server/src/test/resources/lens-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/lens-site.xml b/lens-server/src/test/resources/lens-site.xml
index 4cf94d5..f728810 100644
--- a/lens-server/src/test/resources/lens-site.xml
+++ b/lens-server/src/test/resources/lens-site.xml
@@ -97,7 +97,7 @@
 
   <property>
     <name>lens.server.ws.resourcenames</name>
-    <value>session,metastore,query,quota,scheduler,index,log,test</value>
+    <value>session,metastore,query,quota,scheduler,index,log,test,savedquery</value>
     <description>These JAX-RS resources would be started in the specified order when lens-server starts up</description>
   </property>
 
@@ -108,6 +108,12 @@
   </property>
 
   <property>
+    <name>lens.server.savedquery.ws.resource.impl</name>
+    <value>org.apache.lens.server.query.save.SavedQueryResource</value>
+    <description>Implementation class for saved query Resource</description>
+  </property>
+
+  <property>
     <name>lens.server.user.resolver.type</name>
     <value>FIXED</value>
   </property>
@@ -132,8 +138,14 @@
   </property>
 
   <property>
+    <name>lens.server.savedquery.jdbc.dialectclass</name>
+    <value>org.apache.lens.server.query.save.SavedQueryDao$HSQLDialect</value>
+    <description>Dialect of the target DB, Default is HSQL</description>
+  </property>
+
+  <property>
     <name>lens.server.servicenames</name>
-    <value>session,query,metastore,scheduler,quota,mocknonlens</value>
+    <value>session,query,metastore,scheduler,quota,mocknonlens,savedquery</value>
   </property>
 
   <property>
@@ -143,6 +155,12 @@
   </property>
 
   <property>
+    <name>lens.server.savedquery.service.impl</name>
+    <value>org.apache.lens.server.query.save.SavedQueryServiceImpl</value>
+    <description>Implementation class for saved query service</description>
+  </property>
+
+  <property>
     <name>lens.server.ws.filternames</name>
     <value>authentication,consistentState,serverMode,logging</value>
     <description>These JAX-RS filters would be started in the specified order when lens-server starts up</description>

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/src/site/apt/admin/config.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/admin/config.apt b/src/site/apt/admin/config.apt
index b163a3a..3500441 100644
--- a/src/site/apt/admin/config.apt
+++ b/src/site/apt/admin/config.apt
@@ -147,86 +147,96 @@ Lens server configuration
 *--+--+---+--+
 |59|lens.server.restart.enabled|true|If flag is enabled, all the services will be persisted to persistent location passed.|
 *--+--+---+--+
-|60|lens.server.scheduler.service.impl|org.apache.lens.server.scheduler.QuerySchedulerServiceImpl|Implementation class for query scheduler service|
+|60|lens.server.savedquery.jdbc.dialectclass|org.apache.lens.server.query.save.SavedQueryDao$HSQLDialect|Dialect of the target DB, Default is HSQL. Override with the target DB used.|
 *--+--+---+--+
-|61|lens.server.scheduler.ws.resource.impl|org.apache.lens.server.scheduler.ScheduleResource|Implementation class for query scheduler resource|
+|61|lens.server.savedquery.list.default.count|20|Key denoting the default fetch value of saved query list api.|
 *--+--+---+--+
-|62|lens.server.scheduling.queue.poll.interval.millisec|2000|The interval at which submission thread will poll scheduling queue to fetch the next query for submission. If value is less than equal to 0, then it would mean that thread will continuosly poll without sleeping. The interval has to be given in milliseconds.|
+|62|lens.server.savedquery.list.default.offset|0|Key denoting the default start value of saved query list api.|
 *--+--+---+--+
-|63|lens.server.serverMode.ws.filter.impl|org.apache.lens.server.ServerModeFilter|Implementation class for ServerMode Filter|
+|63|lens.server.savedquery.service.impl|org.apache.lens.server.query.save.SavedQueryServiceImpl|Implementation class for saved query service|
 *--+--+---+--+
-|64|lens.server.service.provider.factory|org.apache.lens.server.ServiceProviderFactoryImpl|Service provider factory implementation class. This parameter is used to lookup the factory implementation class name that would provide an instance of ServiceProvider. Users should instantiate the class to obtain its instance. Example -- Class spfClass = conf.getClass("lens.server.service.provider.factory", null, ServiceProviderFactory.class); ServiceProviderFactory spf = spfClass.newInstance(); ServiceProvider serviceProvider = spf.getServiceProvider(); -- This is not supposed to be overridden by users.|
+|64|lens.server.savedquery.ws.resource.impl|org.apache.lens.server.query.save.SavedQueryResource|Implementation class for Saved query Resource|
 *--+--+---+--+
-|65|lens.server.servicenames|session,query,metastore,scheduler,quota|These services would be started in the specified order when lens-server starts up|
+|65|lens.server.scheduler.service.impl|org.apache.lens.server.scheduler.QuerySchedulerServiceImpl|Implementation class for query scheduler service|
 *--+--+---+--+
-|66|lens.server.session.expiry.service.interval.secs|3600|Interval at which lens session expiry service runs|
+|66|lens.server.scheduler.ws.resource.impl|org.apache.lens.server.scheduler.ScheduleResource|Implementation class for query scheduler resource|
 *--+--+---+--+
-|67|lens.server.session.service.impl|org.apache.lens.server.session.HiveSessionService|Implementation class for session service|
+|67|lens.server.scheduling.queue.poll.interval.millisec|2000|The interval at which submission thread will poll scheduling queue to fetch the next query for submission. If value is less than equal to 0, then it would mean that thread will continuosly poll without sleeping. The interval has to be given in milliseconds.|
 *--+--+---+--+
-|68|lens.server.session.timeout.seconds|86400|Lens session timeout in seconds.If there is no activity on the session for this period then the session will be closed.Default timeout is one day.|
+|68|lens.server.serverMode.ws.filter.impl|org.apache.lens.server.ServerModeFilter|Implementation class for ServerMode Filter|
 *--+--+---+--+
-|69|lens.server.session.ws.resource.impl|org.apache.lens.server.session.SessionResource|Implementation class for Session Resource|
+|69|lens.server.service.provider.factory|org.apache.lens.server.ServiceProviderFactoryImpl|Service provider factory implementation class. This parameter is used to lookup the factory implementation class name that would provide an instance of ServiceProvider. Users should instantiate the class to obtain its instance. Example -- Class spfClass = conf.getClass("lens.server.service.provider.factory", null, ServiceProviderFactory.class); ServiceProviderFactory spf = spfClass.newInstance(); ServiceProvider serviceProvider = spf.getServiceProvider(); -- This is not supposed to be overridden by users.|
 *--+--+---+--+
-|70|lens.server.snapshot.interval|300000|Snapshot interval time in miliseconds for saving lens server state.|
+|70|lens.server.servicenames|session,query,metastore,scheduler,quota,savedquery|These services would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|71|lens.server.state.persist.out.stream.buffer.size|1048576|Output Stream Buffer Size used in writing lens server state to file system. Size is in bytes.|
+|71|lens.server.session.expiry.service.interval.secs|3600|Interval at which lens session expiry service runs|
 *--+--+---+--+
-|72|lens.server.statistics.db|lensstats|Database to which statistics tables are created and partitions are added.|
+|72|lens.server.session.service.impl|org.apache.lens.server.session.HiveSessionService|Implementation class for session service|
 *--+--+---+--+
-|73|lens.server.statistics.log.rollover.interval|3600000|Default rate which log statistics store scans for rollups in milliseconds.|
+|73|lens.server.session.timeout.seconds|86400|Lens session timeout in seconds.If there is no activity on the session for this period then the session will be closed.Default timeout is one day.|
 *--+--+---+--+
-|74|lens.server.statistics.store.class|org.apache.lens.server.stats.store.log.LogStatisticsStore|Default implementation of class used to persist Lens Statistics.|
+|74|lens.server.session.ws.resource.impl|org.apache.lens.server.session.SessionResource|Implementation class for Session Resource|
 *--+--+---+--+
-|75|lens.server.statistics.warehouse.dir|file:///tmp/lens/statistics/warehouse|Default top level location where stats are moved by the log statistics store.|
+|75|lens.server.snapshot.interval|300000|Snapshot interval time in miliseconds for saving lens server state.|
 *--+--+---+--+
-|76|lens.server.total.query.cost.ceiling.per.user|-1.0|A query submitted by user will be launched only if total query cost of all current launched queries of user is less than or equal to total query cost ceiling defined by this property. This configuration value is only useful when TotalQueryCostCeilingConstraint is enabled by using org.apache.lens.server.query.constraint.TotalQueryCostCeilingConstraintFactory as one of the factories in lens.server.query.constraint.factories property. Default is -1.0 which means that there is no limit on the total query cost of launched queries submitted by a user.|
+|76|lens.server.state.persist.out.stream.buffer.size|1048576|Output Stream Buffer Size used in writing lens server state to file system. Size is in bytes.|
 *--+--+---+--+
-|77|lens.server.ui.base.uri|http://0.0.0.0:19999/|The base url for the Lens UI Server|
+|77|lens.server.statistics.db|lensstats|Database to which statistics tables are created and partitions are added.|
 *--+--+---+--+
-|78|lens.server.ui.enable|true|Bringing up the ui server is optional. By default it brings up UI server.|
+|78|lens.server.statistics.log.rollover.interval|3600000|Default rate which log statistics store scans for rollups in milliseconds.|
 *--+--+---+--+
-|79|lens.server.ui.enable.caching|true|Set this to false to disable static file caching in the UI server|
+|79|lens.server.statistics.store.class|org.apache.lens.server.stats.store.log.LogStatisticsStore|Default implementation of class used to persist Lens Statistics.|
 *--+--+---+--+
-|80|lens.server.ui.static.dir|webapp/lens-server/static|The base directory to server UI static files from|
+|80|lens.server.statistics.warehouse.dir|file:///tmp/lens/statistics/warehouse|Default top level location where stats are moved by the log statistics store.|
 *--+--+---+--+
-|81|lens.server.user.resolver.custom.class|full.package.name.Classname|Required for CUSTOM user resolver. In case the provided implementations are not sufficient for user config resolver, a custom classname can be provided. Class should extend org.apache.lens.server.user.UserConfigLoader|
+|81|lens.server.total.query.cost.ceiling.per.user|-1.0|A query submitted by user will be launched only if total query cost of all current launched queries of user is less than or equal to total query cost ceiling defined by this property. This configuration value is only useful when TotalQueryCostCeilingConstraint is enabled by using org.apache.lens.server.query.constraint.TotalQueryCostCeilingConstraintFactory as one of the factories in lens.server.query.constraint.factories property. Default is -1.0 which means that there is no limit on the total query cost of launched queries submitted by a user.|
 *--+--+---+--+
-|82|lens.server.user.resolver.db.keys|lens.session.cluster.user,mapred.job.queue.name|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loaders, the conf keys that will be loaded from database.|
+|82|lens.server.ui.base.uri|http://0.0.0.0:19999/|The base url for the Lens UI Server|
 *--+--+---+--+
-|83|lens.server.user.resolver.db.query|select clusteruser,queue from user_config_table where username=?|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loader, this query will be run with single argument = logged in user and the result columns will be assigned to lens.server.user.resolver.db.keys in order. For ldap backed database resolver, the argument to this query will be the intermediate values obtained from ldap.|
+|83|lens.server.ui.enable|true|Bringing up the ui server is optional. By default it brings up UI server.|
 *--+--+---+--+
-|84|lens.server.user.resolver.fixed.value| |Required for FIXED user resolver. when lens.server.user.resolver.type=FIXED, This will be the value cluster user will resolve to.|
+|84|lens.server.ui.enable.caching|true|Set this to false to disable static file caching in the UI server|
 *--+--+---+--+
-|85|lens.server.user.resolver.ldap.bind.dn| |Required for LDAP_BACKED_DATABASE user resolvers. ldap dn for admin binding example: CN=company-it-admin,ou=service-account,ou=company-service-account,dc=dc1,dc=com...|
+|85|lens.server.ui.static.dir|webapp/lens-server/static|The base directory to server UI static files from|
 *--+--+---+--+
-|86|lens.server.user.resolver.ldap.bind.password| |Required for LDAP_BACKED_DATABASE user resolvers. ldap password for admin binding above|
+|86|lens.server.user.resolver.custom.class|full.package.name.Classname|Required for CUSTOM user resolver. In case the provided implementations are not sufficient for user config resolver, a custom classname can be provided. Class should extend org.apache.lens.server.user.UserConfigLoader|
 *--+--+---+--+
-|87|lens.server.user.resolver.ldap.fields|department|Required for LDAP_BACKED_DATABASE user resolvers. list of fields to be obtained from ldap. These will be cached by the intermediate db.|
+|87|lens.server.user.resolver.db.keys|lens.session.cluster.user,mapred.job.queue.name|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loaders, the conf keys that will be loaded from database.|
 *--+--+---+--+
-|88|lens.server.user.resolver.ldap.intermediate.db.delete.sql|delete from user_department where username=?|Required for LDAP_BACKED_DATABASE user resolvers. query to delete intermediate values from database backing ldap as cache. one argument: logged in user.|
+|88|lens.server.user.resolver.db.query|select clusteruser,queue from user_config_table where username=?|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loader, this query will be run with single argument = logged in user and the result columns will be assigned to lens.server.user.resolver.db.keys in order. For ldap backed database resolver, the argument to this query will be the intermediate values obtained from ldap.|
 *--+--+---+--+
-|89|lens.server.user.resolver.ldap.intermediate.db.insert.sql|insert into user_department (username, department, expiry) values (?, ?, ?)|Required for LDAP_BACKED_DATABASE user resolvers. query to insert intermediate values from database backing ldap as cache. arguments: first logged in user, then all intermediate values, then current time + expiration time|
+|89|lens.server.user.resolver.fixed.value| |Required for FIXED user resolver. when lens.server.user.resolver.type=FIXED, This will be the value cluster user will resolve to.|
 *--+--+---+--+
-|90|lens.server.user.resolver.ldap.intermediate.db.query|select department from user_department where username=? and expiry>?|Required for LDAP_BACKED_DATABASE user resolvers. query to obtain intermediate values from database backing ldap as cache. two arguments: logged in user and current time.|
+|90|lens.server.user.resolver.ldap.bind.dn| |Required for LDAP_BACKED_DATABASE user resolvers. ldap dn for admin binding example: CN=company-it-admin,ou=service-account,ou=company-service-account,dc=dc1,dc=com...|
 *--+--+---+--+
-|91|lens.server.user.resolver.ldap.search.base| |Required for LDAP_BACKED_DATABASE user resolvers. for searching intermediate values for a user, the search keys. example: cn=users,dc=dc1,dc=dc2...|
+|91|lens.server.user.resolver.ldap.bind.password| |Required for LDAP_BACKED_DATABASE user resolvers. ldap password for admin binding above|
 *--+--+---+--+
-|92|lens.server.user.resolver.ldap.search.filter|(&(objectClass=user)(sAMAccountName=%s))|Required for LDAP_BACKED_DATABASE user resolvers. filter pattern for ldap search|
+|92|lens.server.user.resolver.ldap.fields|department|Required for LDAP_BACKED_DATABASE user resolvers. list of fields to be obtained from ldap. These will be cached by the intermediate db.|
 *--+--+---+--+
-|93|lens.server.user.resolver.ldap.url| |Required for LDAP_BACKED_DATABASE user resolvers. ldap url to connect to.|
+|93|lens.server.user.resolver.ldap.intermediate.db.delete.sql|delete from user_department where username=?|Required for LDAP_BACKED_DATABASE user resolvers. query to delete intermediate values from database backing ldap as cache. one argument: logged in user.|
 *--+--+---+--+
-|94|lens.server.user.resolver.propertybased.filename|/path/to/propertyfile|Required for PROPERTYBASED user resolver. when lens.server.user.resolver.type is PROPERTYBASED, then this file will be read and parsed to determine cluster user. Each line should contain username followed by DOT followed by property full name followed by equal-to sign and followed by value. example schema of the file is: user1.lens.server.cluster.user=clusteruser1 user1.mapred.job.queue.name=queue1 *.lens.server.cluster.user=defaultclusteruser *.mapred.job.queue.name=default|
+|94|lens.server.user.resolver.ldap.intermediate.db.insert.sql|insert into user_department (username, department, expiry) values (?, ?, ?)|Required for LDAP_BACKED_DATABASE user resolvers. query to insert intermediate values from database backing ldap as cache. arguments: first logged in user, then all intermediate values, then current time + expiration time|
 *--+--+---+--+
-|95|lens.server.user.resolver.type|FIXED|Type of user config resolver. allowed values are FIXED, PROPERTYBASED, DATABASE, LDAP_BACKED_DATABASE, CUSTOM.|
+|95|lens.server.user.resolver.ldap.intermediate.db.query|select department from user_department where username=? and expiry>?|Required for LDAP_BACKED_DATABASE user resolvers. query to obtain intermediate values from database backing ldap as cache. two arguments: logged in user and current time.|
 *--+--+---+--+
-|96|lens.server.waiting.queries.selection.policy.factories|org.apache.lens.server.query.collect.UserSpecificWaitingQueriesSelectionPolicyFactory|Factories used to instantiate waiting queries selection policies. Every factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.|
+|96|lens.server.user.resolver.ldap.search.base| |Required for LDAP_BACKED_DATABASE user resolvers. for searching intermediate values for a user, the search keys. example: cn=users,dc=dc1,dc=dc2...|
 *--+--+---+--+
-|97|lens.server.ws.featurenames|multipart|These JAX-RS Feature(s) would be started in the specified order when lens-server starts up|
+|97|lens.server.user.resolver.ldap.search.filter|(&(objectClass=user)(sAMAccountName=%s))|Required for LDAP_BACKED_DATABASE user resolvers. filter pattern for ldap search|
 *--+--+---+--+
-|98|lens.server.ws.filternames|authentication,consistentState,serverMode|These JAX-RS filters would be started in the specified order when lens-server starts up|
+|98|lens.server.user.resolver.ldap.url| |Required for LDAP_BACKED_DATABASE user resolvers. ldap url to connect to.|
 *--+--+---+--+
-|99|lens.server.ws.listenernames|appevent|These listeners would be called in the specified order when lens-server starts up|
+|99|lens.server.user.resolver.propertybased.filename|/path/to/propertyfile|Required for PROPERTYBASED user resolver. when lens.server.user.resolver.type is PROPERTYBASED, then this file will be read and parsed to determine cluster user. Each line should contain username followed by DOT followed by property full name followed by equal-to sign and followed by value. example schema of the file is: user1.lens.server.cluster.user=clusteruser1 user1.mapred.job.queue.name=queue1 *.lens.server.cluster.user=defaultclusteruser *.mapred.job.queue.name=default|
 *--+--+---+--+
-|100|lens.server.ws.resourcenames|session,metastore,query,quota,scheduler,index,log|These JAX-RS resources would be started in the specified order when lens-server starts up|
+|100|lens.server.user.resolver.type|FIXED|Type of user config resolver. allowed values are FIXED, PROPERTYBASED, DATABASE, LDAP_BACKED_DATABASE, CUSTOM.|
+*--+--+---+--+
+|101|lens.server.waiting.queries.selection.policy.factories|org.apache.lens.server.query.collect.UserSpecificWaitingQueriesSelectionPolicyFactory|Factories used to instantiate waiting queries selection policies. Every factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.|
+*--+--+---+--+
+|102|lens.server.ws.featurenames|multipart|These JAX-RS Feature(s) would be started in the specified order when lens-server starts up|
+*--+--+---+--+
+|103|lens.server.ws.filternames|authentication,consistentState,serverMode|These JAX-RS filters would be started in the specified order when lens-server starts up|
+*--+--+---+--+
+|104|lens.server.ws.listenernames|appevent|These listeners would be called in the specified order when lens-server starts up|
+*--+--+---+--+
+|105|lens.server.ws.resourcenames|session,metastore,query,quota,scheduler,index,log,savedquery|These JAX-RS resources would be started in the specified order when lens-server starts up|
 *--+--+---+--+
 The configuration parameters and their default values

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/src/site/apt/user/index.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/user/index.apt b/src/site/apt/user/index.apt
index 6a86b1b..812239e 100644
--- a/src/site/apt/user/index.apt
+++ b/src/site/apt/user/index.apt
@@ -191,6 +191,29 @@ Lens User Guide
 
   * {{{./cli.html#Query_Management} CLI query management}}
 
+** Saved queries
+
+  A query can be saved for future execution.
+
+  * Parts of a query could also be parameterised. Values for the parameters can be provided at the time of executing the saved query. If value is not provided, the default value provided at the time of saving the query will be used.
+
+  * Any part of the query could be parameterised. The parameters are mapped with a data type and collection type.
+
+  * During the execution,
+   - STRING parameter will be replaced with a single quoted value
+   - NUMBER, DECIMAL and BOOLEAN will be parsed and resolved (exception will be thrown if the given value is not parsable as the data types mentioned)
+
+  * And collection types,
+	 - SINGLE will be replaced by the simple encoded value
+	 - MULTIPLE will be replaced by (<v1>, <v2>... <vn>)
+	Eg. {select col from table where col = :param}  (param is the parameter)
+	 - If :param is SINGLE and STRING, the query would be resolved to {select col from table where col = '<val>'}
+	 - If :param is SINGLE and NUMBER per se, the query would be resolved to {select col from table where col = 5}
+
+	* A query handle is returned when a saved query is ran.
+
+  * {{{../resource_SavedQueryResource.html}Rest api for saved queries}}
+
 * Metastore service
 
 	The Metastore service is used for DDL operations like creating, updating cubes, fact tables and dimensions. It also pprovides endpoints to create storage tables and to add partitions to a storage table. For more detailed information see the {{{../resource_MetastoreResource.html}metastore service resource}} documentation.