You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by am...@apache.org on 2015/09/15 14:36:21 UTC

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

LENS-742 : Adds query feature : Saved query and parameterization


Project: http://git-wip-us.apache.org/repos/asf/lens/repo
Commit: http://git-wip-us.apache.org/repos/asf/lens/commit/4e81ef4d
Tree: http://git-wip-us.apache.org/repos/asf/lens/tree/4e81ef4d
Diff: http://git-wip-us.apache.org/repos/asf/lens/diff/4e81ef4d

Branch: refs/heads/master
Commit: 4e81ef4ddfef2d8c89c48576e474c2094cbbc56b
Parents: 4addd7b
Author: Amruth S <am...@gmail.com>
Authored: Tue Sep 15 18:05:20 2015 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Tue Sep 15 18:05:20 2015 +0530

----------------------------------------------------------------------
 .../lens/api/error/LensCommonErrorCode.java     |  10 +-
 .../lens/api/query/save/ListResponse.java       |  51 ++
 .../apache/lens/api/query/save/Parameter.java   |  80 +++
 .../api/query/save/ParameterCollectionType.java |  40 ++
 .../lens/api/query/save/ParameterDataType.java  |  48 ++
 .../api/query/save/ParameterParserResponse.java |  50 ++
 .../query/save/ResourceModifiedResponse.java    |  70 +++
 .../apache/lens/api/query/save/SavedQuery.java  |  70 +++
 lens-api/src/main/resources/lens-errors.conf    |  27 ++
 .../lens/server/api/LensConfConstants.java      |  16 +
 .../server/api/query/save/SavedQueryHelper.java |  93 ++++
 .../api/query/save/SavedQueryService.java       |  94 ++++
 .../exception/MissingParameterException.java    |  46 ++
 .../exception/ParameterCollectionException.java |  53 ++
 .../save/exception/ParameterValueException.java |  51 ++
 .../save/exception/PrivilegeException.java      |  51 ++
 .../save/exception/SavedQueryNotFound.java      |  45 ++
 .../save/exception/ValueEncodeException.java    |  49 ++
 .../param/ParameterCollectionTypeEncoder.java   |  70 +++
 .../save/param/ParameterDataTypeEncoder.java    |  91 ++++
 .../api/query/save/param/ParameterParser.java   | 135 ++++++
 .../api/query/save/param/ParameterResolver.java | 126 +++++
 .../api/query/save/TestParameterParser.java     |  79 +++
 .../api/query/save/TestParameterResolution.java | 176 +++++++
 lens-server/enunciate.xml                       |   1 +
 .../lens/server/query/save/SavedQueryApp.java   |  48 ++
 .../lens/server/query/save/SavedQueryDao.java   | 483 +++++++++++++++++++
 .../server/query/save/SavedQueryResource.java   | 307 ++++++++++++
 .../query/save/SavedQueryServiceImpl.java       | 141 ++++++
 .../apache/lens/server/util/UtilityMethods.java |  19 +-
 .../src/main/resources/lensserver-default.xml   |  33 +-
 .../query/save/TestSavedQueryService.java       | 274 +++++++++++
 lens-server/src/test/resources/lens-site.xml    |  22 +-
 src/site/apt/admin/config.apt                   |  92 ++--
 src/site/apt/user/index.apt                     |  23 +
 35 files changed, 3017 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-api/src/main/java/org/apache/lens/api/error/LensCommonErrorCode.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/error/LensCommonErrorCode.java b/lens-api/src/main/java/org/apache/lens/api/error/LensCommonErrorCode.java
index 754e6e1..9c5eaf5 100644
--- a/lens-api/src/main/java/org/apache/lens/api/error/LensCommonErrorCode.java
+++ b/lens-api/src/main/java/org/apache/lens/api/error/LensCommonErrorCode.java
@@ -26,7 +26,15 @@ public enum LensCommonErrorCode {
 
   INTERNAL_SERVER_ERROR(1001),
 
-  INVALID_XML_ERROR(1002);
+  INVALID_XML_ERROR(1002),
+
+  RESOURCE_NOT_FOUND(1003),
+
+  NOT_AUTHORIZED(1004),
+
+  MISSING_PARAMETERS(1005),
+
+  INVALID_PARAMETER_VALUE(1006);
 
   public int getValue() {
     return this.errorCode;

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-api/src/main/java/org/apache/lens/api/query/save/ListResponse.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/query/save/ListResponse.java b/lens-api/src/main/java/org/apache/lens/api/query/save/ListResponse.java
new file mode 100644
index 0000000..e707642
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/query/save/ListResponse.java
@@ -0,0 +1,51 @@
+/**
+ * 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.lens.api.query.save;
+
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+/**
+ * Response of savedqueries list API.
+ */
+@Data
+@AllArgsConstructor
+@NoArgsConstructor
+@XmlRootElement
+public class ListResponse {
+  /**
+   * The starting offset of the result set.
+   */
+  private long offsetAppplied;
+
+  /**
+   * The total number of queries returned in the current request.
+   */
+  private long totalCount;
+
+  /**
+   * The queries.
+   */
+  private List<SavedQuery> resoures;
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-api/src/main/java/org/apache/lens/api/query/save/Parameter.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/query/save/Parameter.java b/lens-api/src/main/java/org/apache/lens/api/query/save/Parameter.java
new file mode 100644
index 0000000..d5f541e
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/query/save/Parameter.java
@@ -0,0 +1,80 @@
+/**
+ * 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.api.query.save;
+
+import javax.xml.bind.Unmarshaller;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.commons.lang.Validate;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.NoArgsConstructor;
+
+/**
+ * The class Parameter.
+ *
+ */
+@Data
+@AllArgsConstructor
+@NoArgsConstructor
+@EqualsAndHashCode
+@XmlRootElement
+public class Parameter {
+  /**
+   * Name of the parameter used in the query.
+   * It should match the following regex pattern ":[a-zA-Z][a-zA-Z_0-9]".
+   */
+  private String name;
+
+  /**
+   * Display name of the parameter. Should be used by the UI apps for resolution.
+   */
+  private String displayName;
+
+  /**
+   * The default value that will be used,
+   * if the parameter is not supplied with any values while running the query.
+   */
+  private String[] defaultValue;
+
+  /**
+   * Data type of the parameter. Could be number, decimal, string or boolean.
+   * The value supplied will be parsed with the corresponding data type.
+   */
+  private ParameterDataType dataType;
+
+  /**
+   * Collection type of the parameter.
+   * Depending on the type of expression IN/EQ, it could be a single/multiple collection.
+   */
+  private ParameterCollectionType collectionType;
+
+  public Parameter(String name) {
+    this.name = name;
+  }
+
+  void afterUnmarshal(Unmarshaller u, Object parent) {
+    Validate.notNull(name);
+    Validate.notNull(displayName);
+    Validate.notNull(dataType);
+    Validate.notNull(collectionType);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterCollectionType.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterCollectionType.java b/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterCollectionType.java
new file mode 100644
index 0000000..971e794
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterCollectionType.java
@@ -0,0 +1,40 @@
+/**
+ * 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.lens.api.query.save;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * The enum ParameterCollectionType
+ * Collection type of a parameter has to be chosen based on its context.
+ * - If it is occurring next to an IN/NOT IN clause, its multiple
+ * - If it is found with EQ/NEQ..>,<,>=,<=,like etc, its single
+ */
+@XmlRootElement
+public enum ParameterCollectionType {
+  /**
+   * Single valued parameter.
+   */
+  SINGLE,
+
+  /**
+   * Multivalued parameter.
+   */
+  MULTIPLE;
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterDataType.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterDataType.java b/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterDataType.java
new file mode 100644
index 0000000..550e89b
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterDataType.java
@@ -0,0 +1,48 @@
+/**
+ * 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.lens.api.query.save;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * The enum ParameterDataType
+ * Should be given based on the column data type.
+ */
+@XmlRootElement
+public enum ParameterDataType {
+  /**
+   * String data type
+   */
+  STRING,
+
+  /**
+   * Number data type
+   */
+  NUMBER,
+
+  /**
+   * Decimal data type
+   */
+  DECIMAL,
+
+  /**
+   * Boolean data type
+   */
+  BOOLEAN;
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterParserResponse.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterParserResponse.java b/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterParserResponse.java
new file mode 100644
index 0000000..aa92f00
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/query/save/ParameterParserResponse.java
@@ -0,0 +1,50 @@
+/**
+ * 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.lens.api.query.save;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import com.google.common.collect.ImmutableSet;
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+/**
+ * Response of parsing a parameterised HQL query
+ */
+@AllArgsConstructor
+@NoArgsConstructor
+@Data
+@XmlRootElement
+public class ParameterParserResponse {
+  /**
+   * Flag that denotes if the query and the parameters are valid
+   */
+  private boolean valid;
+
+  /**
+   * The message
+   */
+  private String message;
+
+  /**
+   * The immutable set of parameters parsed from a parameterised HQL query
+   */
+  private ImmutableSet<Parameter> parameters;
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-api/src/main/java/org/apache/lens/api/query/save/ResourceModifiedResponse.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/query/save/ResourceModifiedResponse.java b/lens-api/src/main/java/org/apache/lens/api/query/save/ResourceModifiedResponse.java
new file mode 100644
index 0000000..4d4f74e
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/query/save/ResourceModifiedResponse.java
@@ -0,0 +1,70 @@
+/**
+ * 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.lens.api.query.save;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+/**
+ * The response of CRUD action on resource.
+ */
+@AllArgsConstructor
+@NoArgsConstructor
+@Data
+@XmlRootElement
+public class ResourceModifiedResponse {
+
+  /**
+   * Action enum that denotes the operation performed
+   */
+  public enum Action {
+    /**
+     * Resource is created
+     */
+    CREATED,
+
+    /**
+     * Resource is updated
+     */
+    UPDATED,
+
+    /**
+     * Resource is updated
+     */
+    DELETED
+  }
+
+  /**
+   * ID of the affected resource
+   */
+  private long id;
+
+  /**
+   * The resource type
+   */
+  private String resource;
+
+  /**
+   * The action performed on the reource
+   */
+  private Action status;
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-api/src/main/java/org/apache/lens/api/query/save/SavedQuery.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/query/save/SavedQuery.java b/lens-api/src/main/java/org/apache/lens/api/query/save/SavedQuery.java
new file mode 100644
index 0000000..c844fe1
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/query/save/SavedQuery.java
@@ -0,0 +1,70 @@
+/**
+ * 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.api.query.save;
+
+import java.util.List;
+
+import javax.xml.bind.Unmarshaller;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.commons.lang.Validate;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+/**
+ * The class representing the saved query
+ */
+@AllArgsConstructor
+@NoArgsConstructor
+@Data
+@XmlRootElement
+public class SavedQuery {
+  /**
+   * ID of the saved query (unique)
+   */
+  private long id;
+
+  /**
+   * Name of the saved query
+   */
+  private String name;
+
+  /**
+   * Description of the saved query
+   */
+  private String description;
+
+  /**
+   * The actual query. Should adhere to HQL syntax
+   */
+  private String query;
+
+  /**
+   * Parameters that are used in the query
+   */
+  private List<Parameter> parameters;
+
+  void afterUnmarshal(Unmarshaller u, Object parent) {
+    Validate.notNull(name);
+    Validate.notNull(query);
+    Validate.notNull(parameters);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-api/src/main/resources/lens-errors.conf
----------------------------------------------------------------------
diff --git a/lens-api/src/main/resources/lens-errors.conf b/lens-api/src/main/resources/lens-errors.conf
index 5428041..a582dc2 100644
--- a/lens-api/src/main/resources/lens-errors.conf
+++ b/lens-api/src/main/resources/lens-errors.conf
@@ -24,6 +24,8 @@
 # Define HTTP status codes
 
 BAD_REQUEST = 400
+NOT_FOUND = 404
+UNAUTHORIZED = 401
 INTERNAL_SERVER_ERROR = 500
 
 # Define all module specific errors
@@ -43,6 +45,31 @@ lensCommonErrors = [
     httpStatusCode = ${BAD_REQUEST}
     errorMsg = "XML invalid: %s"
   }
+
+  {
+    errorCode = 1003
+    httpStatusCode = ${NOT_FOUND}
+    errorMsg = "The specified %s with identifier %s does not exist"
+  }
+
+  {
+    errorCode = 1004
+    httpStatusCode = ${UNAUTHORIZED}
+    errorMsg = "User does not have %s access to %s:%s"
+  }
+
+  {
+    errorCode = 1005
+    httpStatusCode = ${BAD_REQUEST}
+    errorMsg = "These parameters are missing and are required : %s"
+  }
+
+  {
+    errorCode = 1006
+    httpStatusCode = ${BAD_REQUEST}
+    errorMsg = "Value provided {%s} for parameter {%s} is invalid due to {%s}"
+  }
+
 ]
 
 # lensServerErrors: Defined for lens-server module

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java b/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
index fb11f93..90aea0b 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
@@ -926,4 +926,20 @@ public final class LensConfConstants {
    */
   public static final String WAITING_QUERIES_SELECTION_POLICY_FACTORIES_KEY = SERVER_PFX
       + "waiting.queries.selection.policy.factories";
+
+  /**
+   * Key denoting the dialect class property of saved query service.
+   */
+  public static final String JDBC_DIALECT_PROVIDER_CLASS_KEY = "lens.server.savedquery.jdbc.dialectclass";
+
+  /**
+   * Key denoting the default fetch value of saved query list api.
+   */
+  public static final String FETCH_COUNT_SAVED_QUERY_LIST_KEY = "lens.server.savedquery.list.default.count";
+
+  /**
+   * Default fetch count of saved query list api.
+   */
+  public static final int DEFAULT_FETCH_COUNT_SAVED_QUERY_LIST = 20;
+
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/SavedQueryHelper.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/SavedQueryHelper.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/SavedQueryHelper.java
new file mode 100644
index 0000000..04058b6
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/SavedQueryHelper.java
@@ -0,0 +1,93 @@
+/**
+ * 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.lens.server.api.query.save;
+
+import javax.ws.rs.core.MultivaluedHashMap;
+import javax.ws.rs.core.MultivaluedMap;
+
+import org.apache.lens.api.query.save.Parameter;
+import org.apache.lens.api.query.save.ParameterCollectionType;
+import org.apache.lens.api.query.save.SavedQuery;
+import org.apache.lens.server.api.error.LensException;
+import org.apache.lens.server.api.query.save.param.ParameterDataTypeEncoder;
+import org.apache.lens.server.api.query.save.param.ParameterResolver;
+
+import com.google.common.collect.Lists;
+
+public final class SavedQueryHelper {
+
+  private SavedQueryHelper() {
+
+  }
+
+  /**
+   * Extracts the default parameter values from the saved query
+   *
+   * @param query
+   * @return multivalued map of parameter values
+   */
+  public static MultivaluedMap<String, String> getDefaultParameterValues(SavedQuery query) {
+    final MultivaluedMap<String, String> defaults = new MultivaluedHashMap<>();
+    for(Parameter parameter: query.getParameters()) {
+      String[] defaultValues = parameter.getDefaultValue();
+      if (defaultValues != null && defaultValues.length !=0) {
+        defaults.addAll(parameter.getName(), defaultValues);
+      }
+    }
+    return defaults;
+  }
+
+  /**
+   * Gets a sample query for saved query by auto resolving the parameters
+   *
+   * @param savedQuery
+   * @return hql query
+   * @throws LensException
+   */
+  public static String getSampleResolvedQuery(SavedQuery savedQuery) throws LensException {
+    return ParameterResolver.resolve(savedQuery, extrapolateSampleParamValues(savedQuery));
+  }
+
+  /**
+   * Given a saved query, this method extrapolates the parameter values (using the parameter definition)
+   * and returns it as a multivalued map
+   *
+   * @param savedQuery
+   * @return multivalued map containing parameter values
+   */
+  private static MultivaluedMap<String, String> extrapolateSampleParamValues(SavedQuery savedQuery) {
+    final MultivaluedHashMap<String, String> paramValues = new MultivaluedHashMap<>();
+    for(Parameter parameter : savedQuery.getParameters()) {
+      final String sampleValue = ParameterDataTypeEncoder.valueOf(parameter.getDataType().toString()).getSampleValue();
+      if (parameter.getCollectionType() == ParameterCollectionType.SINGLE) {
+        paramValues.putSingle(
+          parameter.getName(),
+          sampleValue
+        );
+      } else if (parameter.getCollectionType() == ParameterCollectionType.MULTIPLE) {
+        paramValues.put(
+          parameter.getName(),
+          Lists.newArrayList(sampleValue, sampleValue)
+        );
+      }
+    }
+    return paramValues;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/SavedQueryService.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/SavedQueryService.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/SavedQueryService.java
new file mode 100644
index 0000000..cba1df6
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/SavedQueryService.java
@@ -0,0 +1,94 @@
+/**
+ * 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.lens.server.api.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.api.error.LensException;
+
+public interface SavedQueryService {
+
+  /**
+   * The Constant NAME.
+   */
+  String NAME = "savedquery";
+
+  /**
+   * Saves a query
+   * @param query  Saved query object.
+   * @return id of the created saved query.
+   * @throws LensException
+   */
+  long save(SavedQuery query) throws LensException;
+
+  /**
+   * Updates the specified saved query with the new object.
+   * @param id     id of the saved query.
+   * @param query  Saved query object.
+   * @throws LensException
+   */
+  void update(long id, SavedQuery query) throws LensException;
+
+  /**
+   * Deletes the saved query specified.
+   * @param id id of the saved query.
+   * @throws LensException
+   */
+  void delete(long id) throws LensException;
+
+  /**
+   * Returns the saved query pointed out by the id.
+   * @param id id of the saved query.
+   * @return saved query object.
+   * @throws LensException
+   */
+  SavedQuery get(long id)  throws LensException;
+
+  /**
+   * List the saved query from {start} to {count} matching filter denoted by criteria.
+   * @param criteria  Multivalued map representing the criteria.
+   * @param start    Displacement from the first matching record.
+   * @param count     Number of records to fetch.
+   * @return list of queries.
+   * @throws LensException
+   */
+  ListResponse list(MultivaluedMap<String, String> criteria, long start, long count) throws LensException;
+
+  /**
+   * Grant permissions for users to do actions on the saved query.
+   * @param id              id of the query.
+   * @param sharingUser     User invoking this action.
+   * @param targetUserPath  Target users who have to get affected.
+   * @param privileges      Privileges to be granted.
+   * @throws LensException
+   */
+  void grant(long id, String sharingUser, String targetUserPath, String[] privileges) throws LensException;
+
+  /**
+   * Revoke permissions from users to do actions on the saved query.
+   * @param id              id of the query.
+   * @param sharingUser     User invoking this action.
+   * @param targetUserPath  Target users who have to get affected.
+   * @param privileges      Privileges to be granted.
+   * @throws LensException
+   */
+  void revoke(long id, String sharingUser, String targetUserPath, String[] privileges) throws LensException;
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/MissingParameterException.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/MissingParameterException.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/MissingParameterException.java
new file mode 100644
index 0000000..14d3298
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/MissingParameterException.java
@@ -0,0 +1,46 @@
+/**
+ * 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.api.query.save.exception;
+
+import static org.apache.lens.api.error.LensCommonErrorCode.MISSING_PARAMETERS;
+
+import java.util.Collection;
+
+import org.apache.lens.server.api.LensErrorInfo;
+import org.apache.lens.server.api.error.LensException;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import lombok.Getter;
+
+/**
+ * The class MissingParameterException. Thrown when a required parameter is not found.
+ */
+public class MissingParameterException extends LensException {
+
+  @Getter
+  private final ImmutableList<String> missingParameters;
+
+  public MissingParameterException(Collection<String> missingParameters) {
+    super(
+      new LensErrorInfo(MISSING_PARAMETERS.getValue(), 0, MISSING_PARAMETERS.toString())
+      , Joiner.on(",").join(missingParameters));
+    this.missingParameters = ImmutableList.copyOf(missingParameters);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ParameterCollectionException.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ParameterCollectionException.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ParameterCollectionException.java
new file mode 100644
index 0000000..332d327
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ParameterCollectionException.java
@@ -0,0 +1,53 @@
+/**
+ * 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.api.query.save.exception;
+
+import java.util.List;
+
+import org.apache.lens.server.api.error.LensException;
+import org.apache.lens.server.api.query.save.param.ParameterCollectionTypeEncoder;
+
+import com.google.common.collect.ImmutableList;
+import lombok.Getter;
+
+/**
+ * The class ParameterCollectionException. Thrown when the collection type of the parameter
+ * and the encoded values are different.
+ */
+public class ParameterCollectionException extends LensException {
+
+  @Getter
+  private final ParameterCollectionTypeEncoder collectionType;
+  @Getter
+  private final ImmutableList<String> values;
+
+  public ParameterCollectionException(
+    ParameterCollectionTypeEncoder collectionType, List<String> values, String message) {
+    super(values + " cannot be encoded as " + collectionType.name() + ", Reason : " + message);
+    this.collectionType = collectionType;
+    this.values = ImmutableList.copyOf(values);
+  }
+
+  public ParameterCollectionException(
+    ParameterCollectionTypeEncoder collectionType, List<String> values, Throwable cause) {
+    super(values + " cannot be encoded as " + collectionType.name(), cause);
+    this.collectionType = collectionType;
+    this.values = ImmutableList.copyOf(values);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ParameterValueException.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ParameterValueException.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ParameterValueException.java
new file mode 100644
index 0000000..49a59ee
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ParameterValueException.java
@@ -0,0 +1,51 @@
+/**
+ * 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.api.query.save.exception;
+
+import static org.apache.lens.api.error.LensCommonErrorCode.INVALID_PARAMETER_VALUE;
+
+import java.util.List;
+
+import org.apache.lens.server.api.LensErrorInfo;
+import org.apache.lens.server.api.error.LensException;
+
+import com.google.common.collect.ImmutableList;
+import lombok.Getter;
+
+/**
+ * The class ParameterValueException.
+ * Thrown when there is an exception encoding the value for a parameter.
+ */
+public class ParameterValueException extends LensException {
+  @Getter
+  private final String paramName;
+  @Getter
+  private final ImmutableList<String> values;
+
+  public ParameterValueException(String paramName, List<String> values, Throwable cause) {
+    super(
+      new LensErrorInfo(INVALID_PARAMETER_VALUE.getValue(), 0, INVALID_PARAMETER_VALUE.toString())
+      , values
+      , paramName
+      , cause.getMessage());
+    this.paramName = paramName;
+    this.values = ImmutableList.copyOf(values);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/PrivilegeException.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/PrivilegeException.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/PrivilegeException.java
new file mode 100644
index 0000000..0d85b8c
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/PrivilegeException.java
@@ -0,0 +1,51 @@
+/**
+ * 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.api.query.save.exception;
+
+import static org.apache.lens.api.error.LensCommonErrorCode.NOT_AUTHORIZED;
+
+import org.apache.lens.server.api.LensErrorInfo;
+import org.apache.lens.server.api.error.LensException;
+
+import lombok.Getter;
+
+/**
+ * The class PrivilegeException. Thrown when the user is
+ * not having the required privileges to complete the action.
+ */
+public class PrivilegeException extends LensException {
+
+  @Getter
+  private final String resourceType;
+  @Getter
+  private final String resourceIdentifier;
+  @Getter
+  private final String privilege;
+
+  public PrivilegeException(String resourceType, String resourceIdentifier, String privilege) {
+    super(
+      new LensErrorInfo(NOT_AUTHORIZED.getValue(), 0, NOT_AUTHORIZED.toString())
+      , privilege
+      , resourceType
+      , resourceIdentifier);
+    this.resourceType = resourceType;
+    this.resourceIdentifier = resourceIdentifier;
+    this.privilege = privilege;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/SavedQueryNotFound.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/SavedQueryNotFound.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/SavedQueryNotFound.java
new file mode 100644
index 0000000..1c5acf8
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/SavedQueryNotFound.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.save.exception;
+
+import static org.apache.lens.api.error.LensCommonErrorCode.RESOURCE_NOT_FOUND;
+
+import org.apache.lens.api.query.save.SavedQuery;
+import org.apache.lens.server.api.LensErrorInfo;
+import org.apache.lens.server.api.error.LensException;
+
+import lombok.Getter;
+
+/**
+ * The class SavedQueryNotFound.
+ * Thrown when the requested saved query is not found.
+ */
+public class SavedQueryNotFound extends LensException {
+
+  @Getter
+  private long id;
+
+  public SavedQueryNotFound(long id) {
+    super(
+      new LensErrorInfo(RESOURCE_NOT_FOUND.getValue(), 0, RESOURCE_NOT_FOUND.toString())
+      , SavedQuery.class.getSimpleName()
+      , String.valueOf(id));
+    this.id = id;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ValueEncodeException.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ValueEncodeException.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ValueEncodeException.java
new file mode 100644
index 0000000..d17e26f
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/exception/ValueEncodeException.java
@@ -0,0 +1,49 @@
+/**
+ * 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.api.query.save.exception;
+
+
+import org.apache.lens.api.query.save.ParameterDataType;
+import org.apache.lens.server.api.error.LensException;
+
+import lombok.Getter;
+
+/**
+ * The class ValueEncodeException.
+ * Thrown when the value cannot be encoded according to the data type specified in the definition.
+ */
+public class ValueEncodeException extends LensException {
+  @Getter
+  private final ParameterDataType dataType;
+  @Getter
+  private final Object valueSupplied;
+
+  public ValueEncodeException(ParameterDataType dataType, Object valueSupplied, Throwable cause) {
+    super(valueSupplied + " cannot be encoded as " + dataType.name(), cause);
+    this.dataType = dataType;
+    this.valueSupplied = valueSupplied;
+  }
+
+  public ValueEncodeException(ParameterDataType dataType, Object valueSupplied) {
+    super(valueSupplied + " cannot be encoded as " + dataType.name());
+    this.dataType = dataType;
+    this.valueSupplied = valueSupplied;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterCollectionTypeEncoder.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterCollectionTypeEncoder.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterCollectionTypeEncoder.java
new file mode 100644
index 0000000..7992eb5
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterCollectionTypeEncoder.java
@@ -0,0 +1,70 @@
+/**
+ * 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.lens.server.api.query.save.param;
+
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.lens.api.query.save.ParameterDataType;
+import org.apache.lens.server.api.query.save.exception.ParameterCollectionException;
+import org.apache.lens.server.api.query.save.exception.ValueEncodeException;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+
+/**
+ * The enum ParameterCollectionTypeEncoder.
+ * Encoder for encoding the value according to the collection type.
+ */
+@XmlRootElement
+public enum ParameterCollectionTypeEncoder {
+  SINGLE {
+    @Override
+    public String encode(ParameterDataType dataType, List<String> values)
+      throws ValueEncodeException, ParameterCollectionException {
+      if (values.size() != 1) {
+        throw new ParameterCollectionException(this, values, "Has to be exactly one");
+      }
+      return ParameterDataTypeEncoder.encodeFor(dataType, values.get(0));
+    }
+  },
+  MULTIPLE {
+    @Override
+    public String encode(final ParameterDataType dataType, List<String> values)
+      throws ValueEncodeException, ParameterCollectionException {
+      if (values.size() <= 0) {
+        throw new ParameterCollectionException(this, values, "Need atleast one value");
+      }
+      final StringBuilder builder = new StringBuilder("(");
+      List<String> transformedValues = Lists.newArrayList();
+      for (String rawValue : values) {
+        transformedValues.add(ParameterDataTypeEncoder.encodeFor(dataType, rawValue));
+      }
+      Joiner.on(',').skipNulls().appendTo(
+        builder,
+        transformedValues
+      );
+      return builder.append(")").toString();
+    }
+  };
+
+  public abstract String encode(ParameterDataType dataType, List<String> value) throws
+    ValueEncodeException, ParameterCollectionException;
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterDataTypeEncoder.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterDataTypeEncoder.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterDataTypeEncoder.java
new file mode 100644
index 0000000..5295f68
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterDataTypeEncoder.java
@@ -0,0 +1,91 @@
+/**
+ * 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.lens.server.api.query.save.param;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.lens.api.query.save.ParameterDataType;
+import org.apache.lens.server.api.query.save.exception.ValueEncodeException;
+
+import org.apache.commons.lang3.StringEscapeUtils;
+
+/**
+ * The enum ParameterDataTypeEncoder.
+ * Encodes the value according to the data type.
+ *
+ */
+@XmlRootElement
+public enum ParameterDataTypeEncoder {
+  STRING {
+    public String encode(String rawValue) {
+      return "'" + StringEscapeUtils.escapeEcmaScript(rawValue) + "'";
+    }
+
+    @Override
+    public String getSampleValue() {
+      return "sample_string_val";
+    }
+  },
+  NUMBER {
+    public String encode(String rawValue) {
+      return String.valueOf(Long.parseLong(rawValue));
+    }
+
+    @Override
+    public String getSampleValue() {
+      return String.valueOf(Long.MAX_VALUE);
+    }
+  },
+  DECIMAL {
+    public String encode(String rawValue) {
+      return String.valueOf(Double.parseDouble(rawValue));
+    }
+
+    @Override
+    public String getSampleValue() {
+      return String.valueOf(Double.MAX_VALUE);
+    }
+  },
+  BOOLEAN {
+    @Override
+    public String encode(String rawValue) {
+      if (rawValue.equalsIgnoreCase("true") || rawValue.equalsIgnoreCase("false")) {
+        return rawValue;
+      }
+      throw new RuntimeException("boolean has to be strictly true or false");
+    }
+
+    @Override
+    public String getSampleValue() {
+      return Boolean.TRUE.toString();
+    }
+  };
+
+  public abstract String encode(String rawValue);
+  public abstract String getSampleValue();
+
+  public static String encodeFor(ParameterDataType dataType, String rawValue) throws ValueEncodeException {
+    try {
+      return ParameterDataTypeEncoder.valueOf(dataType.toString()).encode(rawValue);
+    } catch (Throwable e) {
+      throw new ValueEncodeException(dataType, rawValue , e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterParser.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterParser.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterParser.java
new file mode 100644
index 0000000..7364a2b
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterParser.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.api.query.save.param;
+
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.lens.api.query.save.Parameter;
+import org.apache.lens.api.query.save.ParameterParserResponse;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+/**
+ * Responsible for parsing the query (any grammar) and extracting parameters out of it
+ */
+public class ParameterParser {
+
+  private static final String PARAMETER_INDICATOR_PREFIX = ":";
+  private static final String SINGLE_QUOTED_LITERAL_PATTERN = "'[^']*'";
+  private static final String DOUBLE_QUOTED_LITERAL_PATTERN = "\"[^\"]*\"";
+  private static final Pattern PARAM_PATTERN = Pattern.compile(getPatternStringFor("[a-zA-Z][a-zA-Z1-9_]*"));
+
+  /**
+   *
+   * @param baseParameterPattern, regex denoting the parameters to match
+   * @return a regex pattern that captures the valid occurrences in the 1st group while
+   * ignoring occurrences inside literals (single/double quoted)
+   *
+   * To search for a parameter named p1, we will have to search for ':p1' that
+   * occurs natively in the query ignoring the ones that are single/double quoted.
+   * For eg. In the string - "Hai this is valid param :p1 and 'this is invalid param :p1'", the
+   * first occurrence is valid and the second is invalid as it falls under matching single quotes
+   *
+   * invalid_pattern_1|invalid_pattern_2|PARAMETER_INDICATOR_PREFIX(baseParameterPattern)
+   *
+   * For a baseParameterPattern - param1, the resulting pattern would be
+   * '[^']*'|"[^"]*"|:(param1)
+   */
+  public static String getPatternStringFor(String baseParameterPattern) {
+    final StringBuilder patternBuilder = new StringBuilder();
+    Joiner.on("|").appendTo(patternBuilder, getInvalidPatterns());
+    patternBuilder.append("|");
+    patternBuilder.append(PARAMETER_INDICATOR_PREFIX);
+    patternBuilder.append("(");
+    patternBuilder.append(baseParameterPattern);
+    patternBuilder.append(")");
+    return patternBuilder.toString();
+  }
+
+  /**
+   *
+   * @return a list of invalid regexp patterns that could possibly contain the parameter.
+   */
+  private static List<String> getInvalidPatterns() {
+    final List<String> invalidPatternList = Lists.newArrayList();
+    invalidPatternList.add(SINGLE_QUOTED_LITERAL_PATTERN);
+    invalidPatternList.add(DOUBLE_QUOTED_LITERAL_PATTERN);
+    return invalidPatternList;
+  }
+
+  /**
+   * Returns the length of the parameter prefix configured.
+   *
+   * @return length of the parameter prefix
+   */
+  public static int getParameterPrefixLength() {
+    return PARAMETER_INDICATOR_PREFIX.length();
+  }
+
+  private final String query;
+
+  public ParameterParser(String query) {
+    this.query = query;
+  }
+
+  /**
+   * Returns set of parameter names found in the query
+   *
+   * @return set of parameter names
+   */
+  public Set<String> extractParameterNames() {
+    final ParameterParserResponse parameterParserResponse = extractParameters();
+    return Sets.newHashSet(
+      Collections2.transform(parameterParserResponse.getParameters(), new Function<Parameter, String>() {
+        @Override
+        public String apply(Parameter parameter) {
+          return parameter.getName();
+        }
+      }));
+  }
+
+  /**
+   * Returns parameter parser response for the query
+   *
+   * @return ParameterParserResponse object
+   */
+  public ParameterParserResponse extractParameters() {
+    final Matcher m = PARAM_PATTERN.matcher(query);
+    final Set<Parameter> allParams = Sets.newHashSet();
+    while (m.find()) {
+      String validMatch = m.group(1);
+      if (validMatch != null) {
+        allParams.add(new Parameter(validMatch));
+      }
+    }
+    return new ParameterParserResponse(
+      true,
+      null,
+      ImmutableSet.copyOf(allParams)
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterResolver.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterResolver.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterResolver.java
new file mode 100644
index 0000000..b7a9841
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/save/param/ParameterResolver.java
@@ -0,0 +1,126 @@
+/**
+ * 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.api.query.save.param;
+
+import java.util.List;
+import java.util.Map;
+import java.util.regex.MatchResult;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import javax.ws.rs.core.MultivaluedMap;
+
+import org.apache.lens.api.query.save.Parameter;
+import org.apache.lens.api.query.save.SavedQuery;
+import org.apache.lens.server.api.query.save.SavedQueryHelper;
+import org.apache.lens.server.api.query.save.exception.MissingParameterException;
+import org.apache.lens.server.api.query.save.exception.ParameterCollectionException;
+import org.apache.lens.server.api.query.save.exception.ParameterValueException;
+import org.apache.lens.server.api.query.save.exception.ValueEncodeException;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+
+/**
+ * The class ParameterResolver.
+ * Takes care of resolving the parameter with values specified for the given query
+ *
+ */
+public class ParameterResolver {
+
+  private final ImmutableMap<String, Parameter> parameterMap;
+  private final SavedQuery savedQuery;
+  private final MultivaluedMap<String, String> queryParameters;
+
+  private ParameterResolver(SavedQuery savedQuery, MultivaluedMap<String, String> queryParameters) {
+    final ImmutableMap.Builder<String, Parameter> builder = ImmutableMap.builder();
+    for (Parameter parameter : savedQuery.getParameters()) {
+      builder.put(parameter.getName(), parameter);
+    }
+    parameterMap = builder.build();
+    this.savedQuery = savedQuery;
+    this.queryParameters = SavedQueryHelper.getDefaultParameterValues(savedQuery);
+    this.queryParameters.putAll(queryParameters);
+  }
+
+  /**
+   *
+   * @return resolved query
+   * @throws ParameterValueException, IllegalArgumentException
+   *
+   * The function resolves the provided values for the bind params.
+   * A MissingParameterException is thrown if a value is not provided for one of the params
+   *
+   */
+  private String resolve() throws ParameterValueException, MissingParameterException {
+    final Sets.SetView<String> unProvidedParameters
+      = Sets.difference(parameterMap.keySet(), queryParameters.keySet());
+    if (unProvidedParameters.size() > 0) {
+      throw new MissingParameterException(unProvidedParameters);
+    }
+    final StringBuilder query = new StringBuilder(savedQuery.getQuery());
+    for (Map.Entry<String, List<String>> parameter : queryParameters.entrySet()) {
+      final String parameterName = parameter.getKey();
+      final List<String> values = parameter.getValue();
+      final Parameter parameterDetail = parameterMap.get(parameterName);
+      final String encodedValue;
+      try {
+        encodedValue = ParameterCollectionTypeEncoder
+          .valueOf(parameterDetail.getCollectionType().toString())
+          .encode(
+            parameterDetail.getDataType(),
+            values
+          );
+      } catch (ValueEncodeException | ParameterCollectionException e) {
+        throw new ParameterValueException(parameterName, values, e);
+      }
+      resolveParameter(query, parameterName, encodedValue);
+    }
+    return query.toString();
+  }
+
+  /**
+   * The function replaces all the occurrences of the specified bind parameter with the resolved value.
+   * There could be more than one occurrence of a particular bind param
+   *
+   * For eg. select * from table where col1 = :p1 and col2 = :p1. In this case all the valid occurrences
+   * of the bind param p1 will be replaced by the resolved value.
+   *
+   * @param query (gets mutated)
+   * @param parameter, the bind param
+   * @param resolvedValue, the value to be resolved
+   */
+  private static void resolveParameter(StringBuilder query, String parameter, String resolvedValue) {
+    final Pattern pattern = Pattern.compile(ParameterParser.getPatternStringFor(parameter));
+    Matcher matcher = pattern.matcher(query);
+    while (matcher.find()) {
+      if (matcher.group(1) != null) {
+        final MatchResult result = matcher.toMatchResult();
+        query
+          .replace(result.start(1) - ParameterParser.getParameterPrefixLength(), result.end(1), resolvedValue);
+        matcher = pattern.matcher(query);
+      }
+    }
+  }
+
+  public static String resolve(SavedQuery savedQuery, MultivaluedMap<String, String> queryParameters)
+    throws ParameterValueException, MissingParameterException {
+    return new ParameterResolver(savedQuery, queryParameters).resolve();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/test/java/org/apache/lens/server/api/query/save/TestParameterParser.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/test/java/org/apache/lens/server/api/query/save/TestParameterParser.java b/lens-server-api/src/test/java/org/apache/lens/server/api/query/save/TestParameterParser.java
new file mode 100644
index 0000000..c5c473b
--- /dev/null
+++ b/lens-server-api/src/test/java/org/apache/lens/server/api/query/save/TestParameterParser.java
@@ -0,0 +1,79 @@
+/**
+ * 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.lens.server.api.query.save;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lens.server.api.query.save.param.ParameterParser;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+
+public class TestParameterParser {
+
+  private static final ImmutableMap<String, Set<String>> QUERY_TEST_CASES;
+  static {
+    ImmutableMap.Builder<String, Set<String>> builder = ImmutableMap.builder();
+    builder.put(
+      "select col1 from table where col2 = :param",
+      Sets.newHashSet("param")
+    );
+    builder.put(
+      "select col1 from table where col2 in :param",
+      Sets.newHashSet("param")
+    );
+    builder.put(
+      "select col1 from table where col2 = 'a :param inside single quoted string literal'",
+      Sets.<String>newHashSet()
+    );
+    builder.put(
+      "select col1 from table where col2 = \"a :param inside double quoted string literal\"",
+      Sets.<String>newHashSet()
+    );
+    builder.put(
+      "select col1 from table where col1 = 'value' and col2 = :param and col3 = 'val3'",
+      Sets.newHashSet("param")
+    );
+    builder.put(
+      "select col1 from table where col1 = \"value\" and col2 = :param and col3 = \"val3\"",
+      Sets.newHashSet("param")
+    );
+
+
+    QUERY_TEST_CASES = builder.build();
+  }
+
+
+  @Test
+  public void testParsing() {
+    for(Map.Entry<String, Set<String>> testCase : QUERY_TEST_CASES.entrySet()) {
+      final String query = testCase.getKey();
+      Assert.assertEquals(
+        testCase.getValue(),
+        Sets.newHashSet(new ParameterParser(query).extractParameterNames()),
+        "Test case [[" + testCase.getKey() + "]] failed : "
+      );
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server-api/src/test/java/org/apache/lens/server/api/query/save/TestParameterResolution.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/test/java/org/apache/lens/server/api/query/save/TestParameterResolution.java b/lens-server-api/src/test/java/org/apache/lens/server/api/query/save/TestParameterResolution.java
new file mode 100644
index 0000000..fae65cc
--- /dev/null
+++ b/lens-server-api/src/test/java/org/apache/lens/server/api/query/save/TestParameterResolution.java
@@ -0,0 +1,176 @@
+/**
+ * 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.lens.server.api.query.save;
+
+import static org.apache.lens.api.query.save.ParameterCollectionType.MULTIPLE;
+import static org.apache.lens.api.query.save.ParameterCollectionType.SINGLE;
+import static org.apache.lens.api.query.save.ParameterDataType.*;
+
+import javax.ws.rs.core.MultivaluedHashMap;
+import javax.ws.rs.core.MultivaluedMap;
+
+import org.apache.lens.api.query.save.Parameter;
+import org.apache.lens.api.query.save.SavedQuery;
+import org.apache.lens.server.api.query.save.exception.MissingParameterException;
+import org.apache.lens.server.api.query.save.exception.ParameterValueException;
+import org.apache.lens.server.api.query.save.param.ParameterResolver;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestParameterResolution {
+
+  private static final String QUERY_STRING = "select * from table where "
+    + "col = :param1 and "
+    + "col = :param1 and "
+    + "col = 'a :param1 inside single quotes' and "
+    + "col = \"a :param1 inside double quotes\" and "
+    + "col in :param2 and "
+    + "col = :param3 and "
+    + "col in :param4 and "
+    + "col = :param5 and "
+    + "col = :param7 and "
+    + "col in :param8 and "
+    + "col in :param6";
+
+  private static final SavedQuery QUERY = new SavedQuery(
+    1,
+    "query_name",
+    "description",
+    QUERY_STRING
+    ,
+    Lists.newArrayList(
+      new Parameter(
+        "param1", "Param1", new String[]{"val"}, STRING, SINGLE
+      ),
+      new Parameter(
+        "param2", "Param2", new String[]{"val1", "val2"}, STRING, MULTIPLE
+      ),
+      new Parameter(
+        "param3", "Param3", new String[]{"1"}, NUMBER, SINGLE
+      ),
+      new Parameter(
+        "param4", "Param4", new String[]{"1", "2"}, NUMBER, MULTIPLE
+      ),
+      new Parameter(
+        "param7", "Param7", new String[]{"1"}, DECIMAL, SINGLE
+      ),
+      new Parameter(
+        "param8", "Param8", new String[]{"1.2", "2.1"}, DECIMAL, MULTIPLE
+      ),
+      new Parameter(
+        "param5", "Param5", new String[]{"true"}, BOOLEAN, SINGLE
+      ),
+      new Parameter(
+        "param6", "Param6", null, BOOLEAN, MULTIPLE
+      )
+    )
+  );
+
+
+  @Test
+  public void testWithProperValues() throws ParameterValueException, MissingParameterException {
+    MultivaluedMap<String, String> parameterValues = new MultivaluedHashMap<>();
+    parameterValues.put("param6", Lists.newArrayList("true", "false"));
+    String resolvedQuery = ParameterResolver.resolve(QUERY, parameterValues);
+    Assert.assertEquals(
+      "select * from table where col = 'val' and col = 'val' and col = 'a :param1 inside single quotes' "
+        + "and col = \"a :param1 inside double quotes\" and col in ('val1','val2') and col = 1 and col in (1,2) "
+        + "and col = true and col = 1.0 and col in (1.2,2.1) and col in (true,false)",
+      resolvedQuery,
+      "Query resolution did not happen correctly"
+    );
+  }
+
+  @Test
+  public void testWithInvalidValueForNumber() throws MissingParameterException {
+    MultivaluedMap<String, String> parameterValues = new MultivaluedHashMap<>();
+    parameterValues.put("param3", Lists.newArrayList("number"));
+    parameterValues.put("param6", Lists.newArrayList("true", "false"));
+    try {
+      final String resolvedQuery = ParameterResolver.resolve(QUERY, parameterValues);
+      Assert.fail("The query seem to have resolved with invalid datatype : " + resolvedQuery);
+    } catch (ParameterValueException e) {
+      Assert.assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testWithInvalidValueForBoolean() throws MissingParameterException {
+    MultivaluedMap<String, String> parameterValues = new MultivaluedHashMap<>();
+    parameterValues.put("param5", Lists.newArrayList("boolean"));
+    parameterValues.put("param6", Lists.newArrayList("true", "false"));
+    try {
+      final String resolvedQuery = ParameterResolver.resolve(QUERY, parameterValues);
+      Assert.fail("The query seem to have resolved with invalid datatype : " + resolvedQuery);
+    } catch (ParameterValueException e) {
+      Assert.assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testWithInvalidValueForDecimal() throws MissingParameterException {
+    MultivaluedMap<String, String> parameterValues = new MultivaluedHashMap<>();
+    parameterValues.put("param7", Lists.newArrayList("decimal"));
+    parameterValues.put("param6", Lists.newArrayList("true", "false"));
+    try {
+      final String resolvedQuery = ParameterResolver.resolve(QUERY, parameterValues);
+      Assert.fail("The query seem to have resolved with invalid datatype : " + resolvedQuery);
+    } catch (ParameterValueException e) {
+      Assert.assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testWithIncorrectCollectionTypeForSingle() throws MissingParameterException {
+    MultivaluedMap<String, String> parameterValues = new MultivaluedHashMap<>();
+    parameterValues.put("param5", Lists.newArrayList("true", "false"));
+    parameterValues.put("param6", Lists.newArrayList("true"));
+    try {
+      final String resolvedQuery = ParameterResolver.resolve(QUERY, parameterValues);
+      Assert.fail("The query seem to have resolved with invalid collection type : " + resolvedQuery);
+    } catch (ParameterValueException e) {
+      Assert.assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testWithIncorrectCollectionTypeForMultiple() throws MissingParameterException {
+    MultivaluedMap<String, String> parameterValues = new MultivaluedHashMap<>();
+    parameterValues.put("param6", Lists.<String>newArrayList());
+    try {
+      final String resolvedQuery = ParameterResolver.resolve(QUERY, parameterValues);
+      Assert.fail("The query seem to have resolved with invalid collection type : " + resolvedQuery);
+    } catch (ParameterValueException e) {
+      Assert.assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testWithMissingParameters() throws ParameterValueException{
+    try {
+      String resolvedQuery = ParameterResolver.resolve(QUERY, new MultivaluedHashMap<String, String>());
+      Assert.fail("The query seem to have resolved with missing parameters : " + resolvedQuery);
+    } catch (MissingParameterException e) {
+      Assert.assertTrue(true);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server/enunciate.xml
----------------------------------------------------------------------
diff --git a/lens-server/enunciate.xml b/lens-server/enunciate.xml
index 94b5199..7013f13 100644
--- a/lens-server/enunciate.xml
+++ b/lens-server/enunciate.xml
@@ -31,6 +31,7 @@
   </services>
   <api-classes>
     <include pattern="org.apache.lens.server.query.*"/>
+    <include pattern="org.apache.lens.server.query.save.*"/>
     <include pattern="org.apache.lens.server.metastore.*"/>
     <include pattern="org.apache.lens.server.quota.*"/>
     <include pattern="org.apache.lens.server.scheduler.*"/>

http://git-wip-us.apache.org/repos/asf/lens/blob/4e81ef4d/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryApp.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryApp.java b/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryApp.java
new file mode 100644
index 0000000..e55ed13
--- /dev/null
+++ b/lens-server/src/main/java/org/apache/lens/server/query/save/SavedQueryApp.java
@@ -0,0 +1,48 @@
+/**
+ * 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.util.HashSet;
+import java.util.Set;
+
+import javax.ws.rs.ApplicationPath;
+import javax.ws.rs.core.Application;
+
+import org.apache.lens.server.LensApplicationListener;
+import org.apache.lens.server.LensRequestContextInitFilter;
+
+import org.glassfish.jersey.filter.LoggingFilter;
+import org.glassfish.jersey.media.multipart.MultiPartFeature;
+
+/**
+ * The Class SavedQueryApp.
+ */
+@ApplicationPath("/savedquery")
+public class SavedQueryApp extends Application {
+  @Override
+  public Set<Class<?>> getClasses() {
+    final Set<Class<?>> classes = new HashSet<Class<?>>();
+    classes.add(SavedQueryResource.class);
+    classes.add(MultiPartFeature.class);
+    classes.add(LensRequestContextInitFilter.class);
+    classes.add(LoggingFilter.class);
+    classes.add(LensApplicationListener.class);
+    return classes;
+  }
+}