You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by pr...@apache.org on 2016/09/15 08:55:56 UTC

lens git commit: LENS-1319: Add option to pass query conf while submitting query via Lens Java Client

Repository: lens
Updated Branches:
  refs/heads/master 3ab732acc -> 12774400d


LENS-1319: Add option to pass query conf while submitting query via Lens Java Client


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

Branch: refs/heads/master
Commit: 12774400d449e276991cc08539456e769630b3ac
Parents: 3ab732a
Author: Srikanth Sundarrajan <sr...@hotmail.com>
Authored: Thu Sep 15 14:23:17 2016 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Sep 15 14:23:17 2016 +0530

----------------------------------------------------------------------
 .../lens/cli/commands/LensQueryCommands.java    |  10 +-
 .../apache/lens/cli/TestLensQueryCommands.java  |   4 +-
 .../java/org/apache/lens/client/LensClient.java |  51 ++++++-
 .../org/apache/lens/client/LensStatement.java   | 134 +++++++++++++++++--
 .../lens/client/jdbc/LensJdbcStatement.java     |   5 +-
 .../org/apache/lens/client/TestLensClient.java  |  21 ++-
 .../org/apache/lens/examples/SampleQueries.java |   3 +-
 7 files changed, 197 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/12774400/lens-cli/src/main/java/org/apache/lens/cli/commands/LensQueryCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/main/java/org/apache/lens/cli/commands/LensQueryCommands.java b/lens-cli/src/main/java/org/apache/lens/cli/commands/LensQueryCommands.java
index 4346c86..4b99495 100644
--- a/lens-cli/src/main/java/org/apache/lens/cli/commands/LensQueryCommands.java
+++ b/lens-cli/src/main/java/org/apache/lens/cli/commands/LensQueryCommands.java
@@ -25,6 +25,7 @@ import java.util.UUID;
 
 import javax.ws.rs.core.Response;
 
+import org.apache.lens.api.LensConf;
 import org.apache.lens.api.query.*;
 import org.apache.lens.api.result.PrettyPrintable;
 import org.apache.lens.cli.commands.annotations.UserDocumentation;
@@ -110,14 +111,15 @@ public class LensQueryCommands extends BaseLensCommand {
 
     try {
       if (async) {
-        QueryHandle queryHandle = getClient().executeQueryAsynch(sql, queryName);
+        QueryHandle queryHandle = getClient().executeQueryAsynch(sql, queryName, new LensConf());
         return queryHandle.getHandleIdString();
       } else {
         LensClientResultSetWithStats resultWithStats;
         long timeOutMillis = getClient().getConf().getLong(LensCliConfigConstants.QUERY_EXECUTE_TIMEOUT_MILLIS,
             LensCliConfigConstants.DEFAULT_QUERY_EXECUTE_TIMEOUT_MILLIS);
         LensClient.getCliLogger().info("Executing query with timeout of {} milliseconds", timeOutMillis);
-        QueryHandleWithResultSet result = getClient().executeQueryWithTimeout(sql, queryName, timeOutMillis);
+        QueryHandleWithResultSet result = getClient().executeQueryWithTimeout(sql, queryName, timeOutMillis,
+          new LensConf());
         if (result.getResult() == null) {
           //Query not finished yet. Wait till it finishes and get result.
           LensClient.getCliLogger().info("Couldn't complete query execution within timeout. Waiting for completion");
@@ -475,7 +477,7 @@ public class LensQueryCommands extends BaseLensCommand {
   public String prepare(@CliOption(key = {"", "query"}, mandatory = true, help = "<query-string>") String sql,
     @CliOption(key = {"name"}, mandatory = false, help = "<query-name>") String queryName)
     throws UnsupportedEncodingException, LensAPIException {
-    return getClient().prepare(sql, queryName).getData().toString();
+    return getClient().prepare(sql, queryName, new LensConf()).getData().toString();
   }
 
   /**
@@ -499,7 +501,7 @@ public class LensQueryCommands extends BaseLensCommand {
     throws UnsupportedEncodingException, LensAPIException {
     PrettyPrintable cliOutput;
     try {
-      QueryPlan plan = getClient().explainAndPrepare(sql, queryName).getData();
+      QueryPlan plan = getClient().explainAndPrepare(sql, queryName, new LensConf()).getData();
       return plan.getPlanString() + "\n" + "Prepare handle:" + plan.getPrepareHandle();
     } catch (final LensAPIException e) {
       BriefError briefError = new BriefError(e.getLensAPIErrorCode(), e.getLensAPIErrorMessage());

http://git-wip-us.apache.org/repos/asf/lens/blob/12774400/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
index e1b19de..e75fc0e 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
@@ -32,6 +32,7 @@ import javax.ws.rs.BadRequestException;
 import javax.xml.datatype.DatatypeFactory;
 
 import org.apache.lens.api.APIResult;
+import org.apache.lens.api.LensConf;
 import org.apache.lens.api.metastore.*;
 import org.apache.lens.api.query.LensQuery;
 import org.apache.lens.api.query.QueryHandle;
@@ -534,7 +535,8 @@ public class TestLensQueryCommands extends LensCliApplicationTest {
   @Test
   public void testProxyLensQuery() throws Exception {
     LensClient client = new LensClient();
-    QueryHandle handle = client.executeQueryAsynch("cube select id,name from test_dim", "proxyTestQuery");
+    QueryHandle handle = client.executeQueryAsynch("cube select id,name from test_dim", "proxyTestQuery",
+      new LensConf());
     client.getStatement().waitForQueryToComplete(handle);
     LensQuery query = client.getQueryDetails(handle);
     ProxyLensQuery proxyQuery = new ProxyLensQuery(client.getStatement(), handle);

http://git-wip-us.apache.org/repos/asf/lens/blob/12774400/lens-client/src/main/java/org/apache/lens/client/LensClient.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/LensClient.java b/lens-client/src/main/java/org/apache/lens/client/LensClient.java
index 9a76118..593cc08 100644
--- a/lens-client/src/main/java/org/apache/lens/client/LensClient.java
+++ b/lens-client/src/main/java/org/apache/lens/client/LensClient.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import javax.ws.rs.core.Response;
 
 import org.apache.lens.api.APIResult;
+import org.apache.lens.api.LensConf;
 import org.apache.lens.api.metastore.*;
 import org.apache.lens.api.query.*;
 import org.apache.lens.api.result.LensAPIResult;
@@ -112,9 +113,14 @@ public class LensClient implements AutoCloseable {
     return mc;
   }
 
+  @Deprecated
   public QueryHandle executeQueryAsynch(String sql, String queryName) throws LensAPIException {
+    return executeQueryAsynch(sql, queryName, new LensConf());
+  }
+
+  public QueryHandle executeQueryAsynch(String sql, String queryName, LensConf conf) throws LensAPIException {
     log.debug("Executing query {}", sql);
-    QueryHandle handle = statement.executeQuery(sql, false, queryName);
+    QueryHandle handle = statement.executeQuery(sql, false, queryName, conf);
     statementMap.put(handle, statement);
     return handle;
   }
@@ -130,10 +136,28 @@ public class LensClient implements AutoCloseable {
    * @return
    * @throws LensAPIException
    */
+  @Deprecated
   public QueryHandleWithResultSet executeQueryWithTimeout(String sql, String queryName, long timeOutMillis)
     throws LensAPIException {
+    return executeQueryWithTimeout(sql, queryName, timeOutMillis, new LensConf());
+  }
+
+  /**
+   * Execute query with timeout option.
+   * If the query does not finish within the timeout time, server returns the query handle which can be used to
+   * track further progress.
+   *
+   * @param sql : query/command to be executed
+   * @param queryName : optional query name
+   * @param timeOutMillis : timeout milliseconds for the query execution.
+   * @param conf      config to be used for the query
+   * @return
+   * @throws LensAPIException
+   */
+  public QueryHandleWithResultSet executeQueryWithTimeout(String sql, String queryName,
+    long timeOutMillis, LensConf conf) throws LensAPIException {
     log.info("Executing query {} with timeout of {} milliseconds", sql, timeOutMillis);
-    QueryHandleWithResultSet result = statement.executeQuery(sql, queryName, timeOutMillis);
+    QueryHandleWithResultSet result = statement.executeQuery(sql, queryName, timeOutMillis, conf);
     statementMap.put(result.getQueryHandle(), statement);
     if (result.getStatus().failed()) {
       IdBriefErrorTemplate errorResult = new IdBriefErrorTemplate(IdBriefErrorTemplateKey.QUERY_ID,
@@ -289,7 +313,7 @@ public class LensClient implements AutoCloseable {
   }
 
   public LensAPIResult<QueryPlan> getQueryPlan(String q) throws LensAPIException {
-    return statement.explainQuery(q);
+    return statement.explainQuery(q, new LensConf());
   }
 
   public boolean killQuery(QueryHandle q) {
@@ -651,11 +675,23 @@ public class LensClient implements AutoCloseable {
     return mc.updatePartitionsOfDimensionTable(table, storage, partsSpec);
   }
 
+  @Deprecated
   public LensAPIResult<QueryPrepareHandle> prepare(String sql, String queryName) throws LensAPIException {
-    return statement.prepareQuery(sql, queryName);
+    return prepare(sql, queryName, new LensConf());
+  }
+
+  public LensAPIResult<QueryPrepareHandle> prepare(String sql, String queryName, LensConf conf)
+    throws LensAPIException {
+    return statement.prepareQuery(sql, queryName, conf);
   }
 
+  @Deprecated
   public LensAPIResult<QueryPlan> explainAndPrepare(String sql, String queryName) throws LensAPIException {
+    return explainAndPrepare(sql, queryName, new LensConf());
+  }
+
+  public LensAPIResult<QueryPlan> explainAndPrepare(String sql, String queryName, LensConf conf)
+    throws LensAPIException {
     return statement.explainAndPrepare(sql, queryName);
   }
 
@@ -677,8 +713,13 @@ public class LensClient implements AutoCloseable {
     return getResultsFromHandle(qh, true);
   }
 
+  @Deprecated
   public QueryHandle executePrepared(QueryPrepareHandle phandle, String queryName) {
-    return statement.executeQuery(phandle, false, queryName);
+    return executePrepared(phandle, queryName, new LensConf());
+  }
+
+  public QueryHandle executePrepared(QueryPrepareHandle phandle, String queryName, LensConf conf) {
+    return statement.executeQuery(phandle, false, queryName, conf);
   }
 
   public boolean isConnectionOpen() {

http://git-wip-us.apache.org/repos/asf/lens/blob/12774400/lens-client/src/main/java/org/apache/lens/client/LensStatement.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/LensStatement.java b/lens-client/src/main/java/org/apache/lens/client/LensStatement.java
index 64c24c7..0c0cf91 100644
--- a/lens-client/src/main/java/org/apache/lens/client/LensStatement.java
+++ b/lens-client/src/main/java/org/apache/lens/client/LensStatement.java
@@ -70,10 +70,29 @@ public class LensStatement {
    * @param queryName              the query name
    * @return the query handle
    */
+  @Deprecated
   public QueryHandle executeQuery(String sql, boolean waitForQueryToComplete, String queryName)
     throws LensAPIException {
+    return executeQuery(sql, waitForQueryToComplete, queryName, new LensConf());
+  }
+
+  /**
+   * This method can be used for executing a query. If waitForQueryToComplete is false, the call to this method returns
+   * immediately after submitting the query to the server without waiting for it to complete execution.
+   * <p>
+   * {@link #getStatus(QueryHandle)} can be used to track to track the query progress and
+   * {@link #getQuery(QueryHandle)} can be used to get complete details (including status) about the query.
+   *
+   * @param sql                    the sql
+   * @param waitForQueryToComplete the wait for query to complete
+   * @param queryName              the query name
+   * @param conf                   config specific to this query
+   * @return the query handle
+   */
+  public QueryHandle executeQuery(String sql, boolean waitForQueryToComplete, String queryName, LensConf conf)
+    throws LensAPIException {
 
-    QueryHandle handle = submitQuery(sql, queryName);
+    QueryHandle handle = submitQuery(sql, queryName, conf);
 
     if (waitForQueryToComplete) {
       waitForQueryToComplete(handle);
@@ -93,8 +112,27 @@ public class LensStatement {
    * @param queryName              the query name
    * @return the query handle
    */
+  @Deprecated
   public QueryHandle executeQuery(QueryPrepareHandle phandle, boolean waitForQueryToComplete, String queryName) {
-    QueryHandle handle = submitQuery(phandle, queryName);
+    return executeQuery(phandle, waitForQueryToComplete, queryName, new LensConf());
+  }
+
+  /**
+   * This method can be used for executing a prepared query. If waitForQueryToComplete is false, the call to this method
+   * returns immediately after submitting the query to the server without waiting for it to complete execution.
+   * <p>
+   * {@link #getStatus(QueryHandle)} can be used to track to track the query progress and
+   * {@link #getQuery(QueryHandle)} can be used to get complete details (including status) about the query.
+   *
+   * @param phandle                the prepared query handle
+   * @param waitForQueryToComplete the wait for query to complete
+   * @param queryName              the query name
+   * @param conf                   config to be used for the query
+   * @return the query handle
+   */
+  public QueryHandle executeQuery(QueryPrepareHandle phandle, boolean waitForQueryToComplete, String queryName,
+     LensConf conf) {
+    QueryHandle handle = submitQuery(phandle, queryName, conf);
 
     if (waitForQueryToComplete) {
       waitForQueryToComplete(handle);
@@ -121,8 +159,34 @@ public class LensStatement {
    * @return QueryHandleWithResultSet
    * @throws LensAPIException
    */
+  @Deprecated
   public QueryHandleWithResultSet executeQuery(String sql, String queryName, long timeOutMillis)
     throws LensAPIException {
+    return executeQuery(sql, queryName, timeOutMillis, new LensConf());
+  }
+
+  /**
+   * This method can be used for executing query. The method waits for timeOutMillis time OR query execution to succeed,
+   * which ever happens first, before returning the response to the caller.
+   * <p>
+   * If the query execution finishes before timeout, user can check the query Status (SUCCESSFUL/FAILED) using
+   * {@link QueryHandleWithResultSet#getStatus()} and access the result of SUCCESSFUL query via
+   * {@link QueryHandleWithResultSet#getResult()} and {@link QueryHandleWithResultSet#getResultMetadata()}.
+   * <p>
+   * If the query does not finish within the timeout, user can use {@link #getStatus(QueryHandle)} to track
+   * the query progress and {@link #getQuery(QueryHandle)} to get complete details (including status) about
+   * the query. Once the query has reached SUCCESSFUL state, user can access the results via
+   * {@link #getResultSet(LensQuery)} and {@link #getResultSetMetaData(LensQuery)}
+   *
+   * @param sql : query/command to be executed
+   * @param queryName : optional query name
+   * @param timeOutMillis : timeout milliseconds
+   * @param conf      config to be used for the query
+   * @return QueryHandleWithResultSet
+   * @throws LensAPIException
+   */
+  public QueryHandleWithResultSet executeQuery(String sql, String queryName, long timeOutMillis, LensConf conf)
+    throws LensAPIException {
     if (!connection.isOpen()) {
       throw new IllegalStateException("Lens Connection has to be established before querying");
     }
@@ -136,7 +200,7 @@ public class LensStatement {
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("timeoutmillis").build(), "" + timeOutMillis));
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("queryName").build(), queryName == null ? ""
         : queryName));
-    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
+    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
         MediaType.APPLICATION_XML_TYPE));
     WebTarget target = getQueryWebTarget(client);
 
@@ -161,7 +225,21 @@ public class LensStatement {
    * @return the query prepare handle
    * @throws LensAPIException
    */
+  @Deprecated
   public LensAPIResult<QueryPrepareHandle> prepareQuery(String sql, String queryName) throws LensAPIException {
+    return prepareQuery(sql, queryName, new LensConf());
+  }
+
+  /**
+   * Prepare query.
+   *
+   * @param sql       the sql
+   * @param queryName the query name
+   * @return the query prepare handle
+   * @throws LensAPIException
+   */
+  public LensAPIResult<QueryPrepareHandle> prepareQuery(String sql, String queryName,
+     LensConf conf) throws LensAPIException {
     if (!connection.isOpen()) {
       throw new IllegalStateException("Lens Connection has to be established before querying");
     }
@@ -169,7 +247,7 @@ public class LensStatement {
     Client client = connection.buildClient();
     WebTarget target = getPreparedQueriesWebTarget(client);
 
-    Response response = target.request().post(Entity.entity(prepareForm(sql, "PREPARE", queryName),
+    Response response = target.request().post(Entity.entity(prepareForm(sql, "PREPARE", queryName, conf),
         MediaType.MULTIPART_FORM_DATA_TYPE));
 
     if (response.getStatus() == Response.Status.OK.getStatusCode()) {
@@ -187,7 +265,22 @@ public class LensStatement {
    * @return the query plan
    * @throws LensAPIException
    */
+  @Deprecated
   public LensAPIResult<QueryPlan> explainAndPrepare(String sql, String queryName) throws LensAPIException {
+    return explainAndPrepare(sql, queryName, new LensConf());
+  }
+
+  /**
+   * Explain and prepare.
+   *
+   * @param sql       the sql
+   * @param queryName the query name
+   * @param conf      config to be used for the query
+   * @return the query plan
+   * @throws LensAPIException
+   */
+  public LensAPIResult<QueryPlan> explainAndPrepare(String sql, String queryName,
+    LensConf conf) throws LensAPIException {
     if (!connection.isOpen()) {
       throw new IllegalStateException("Lens Connection has to be established before querying");
     }
@@ -197,7 +290,7 @@ public class LensStatement {
     WebTarget target = getPreparedQueriesWebTarget(client);
 
     Response response = target.request().post(
-        Entity.entity(prepareForm(sql, "EXPLAIN_AND_PREPARE", queryName), MediaType.MULTIPART_FORM_DATA_TYPE),
+        Entity.entity(prepareForm(sql, "EXPLAIN_AND_PREPARE", queryName, conf), MediaType.MULTIPART_FORM_DATA_TYPE),
         Response.class);
     if (response.getStatus() == Response.Status.OK.getStatusCode()) {
       return response.readEntity(new GenericType<LensAPIResult<QueryPlan>>() {});
@@ -213,9 +306,10 @@ public class LensStatement {
    * @param sql       the sql
    * @param op        the op
    * @param queryName the query name
+   * @param conf      config to be used for the query
    * @return the form data multi part
    */
-  private FormDataMultiPart prepareForm(String sql, String op, String queryName) {
+  private FormDataMultiPart prepareForm(String sql, String op, String queryName, LensConf conf) {
     FormDataMultiPart mp = new FormDataMultiPart();
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionid").build(), connection
       .getSessionHandle(), MediaType.APPLICATION_XML_TYPE));
@@ -224,7 +318,7 @@ public class LensStatement {
     if (!StringUtils.isBlank(queryName)) {
       mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("queryName").build(), queryName));
     }
-    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
+    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
       MediaType.APPLICATION_XML_TYPE));
     return mp;
   }
@@ -357,9 +451,10 @@ public class LensStatement {
    *
    * @param sql       the sql
    * @param queryName the query name
+   * @param conf      config to be used for the query
    * @return the query handle
    */
-  private QueryHandle submitQuery(String sql, String queryName) throws LensAPIException {
+  private QueryHandle submitQuery(String sql, String queryName, LensConf conf) throws LensAPIException {
     if (!connection.isOpen()) {
       throw new IllegalStateException("Lens Connection has to be established before querying");
     }
@@ -372,7 +467,7 @@ public class LensStatement {
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("operation").build(), "EXECUTE"));
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("queryName").build(), queryName == null ? ""
       : queryName));
-    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
+    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
       MediaType.APPLICATION_XML_TYPE));
     WebTarget target = getQueryWebTarget(client);
 
@@ -392,10 +487,10 @@ public class LensStatement {
    *
    * @param phandle   the phandle
    * @param queryName the query name
+   * @param conf      config to be used for the query
    * @return the query handle
-   * @throws LensAPIException
    */
-  private QueryHandle submitQuery(QueryPrepareHandle phandle, String queryName) {
+  private QueryHandle submitQuery(QueryPrepareHandle phandle, String queryName, LensConf conf) {
     if (!connection.isOpen()) {
       throw new IllegalStateException("Lens Connection has to be " + "established before querying");
     }
@@ -408,7 +503,7 @@ public class LensStatement {
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("operation").build(), "execute"));
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("queryName").build(), queryName == null ? ""
       : queryName));
-    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
+    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
       MediaType.APPLICATION_XML_TYPE));
     QueryHandle handle = target.request()
         .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE), QueryHandle.class);
@@ -424,7 +519,20 @@ public class LensStatement {
    * @return the query plan
    * @throws LensAPIException
    */
+  @Deprecated
   public LensAPIResult<QueryPlan> explainQuery(String sql) throws LensAPIException {
+    return explainQuery(sql, new LensConf());
+  }
+
+  /**
+   * Explain query.
+   *
+   * @param sql   the sql
+   * @param conf  config to be used for the query
+   * @return the query plan
+   * @throws LensAPIException
+   */
+  public LensAPIResult<QueryPlan> explainQuery(String sql, LensConf conf) throws LensAPIException {
     if (!connection.isOpen()) {
       throw new IllegalStateException("Lens Connection has to be established before querying");
     }
@@ -435,7 +543,7 @@ public class LensStatement {
       .getSessionHandle(), MediaType.APPLICATION_XML_TYPE));
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("query").build(), sql));
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("operation").build(), "explain"));
-    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
+    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
       MediaType.APPLICATION_XML_TYPE));
     WebTarget target = getQueryWebTarget(client);
 

http://git-wip-us.apache.org/repos/asf/lens/blob/12774400/lens-client/src/main/java/org/apache/lens/client/jdbc/LensJdbcStatement.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/jdbc/LensJdbcStatement.java b/lens-client/src/main/java/org/apache/lens/client/jdbc/LensJdbcStatement.java
index fb265e5..5888394 100644
--- a/lens-client/src/main/java/org/apache/lens/client/jdbc/LensJdbcStatement.java
+++ b/lens-client/src/main/java/org/apache/lens/client/jdbc/LensJdbcStatement.java
@@ -20,6 +20,7 @@ package org.apache.lens.client.jdbc;
 
 import java.sql.*;
 
+import org.apache.lens.api.LensConf;
 import org.apache.lens.client.LensStatement;
 import org.apache.lens.client.exceptions.LensAPIException;
 
@@ -58,7 +59,7 @@ public class LensJdbcStatement implements Statement {
   @Override
   public ResultSet executeQuery(String s) throws SQLException {
     try {
-      statement.executeQuery(s, true, null);
+      statement.executeQuery(s, true, null, new LensConf());
     } catch (LensAPIException e) {
       log.error("Execution Failed for Statement:{}", s, e);
     }
@@ -115,7 +116,7 @@ public class LensJdbcStatement implements Statement {
       throw new SQLException("Cannot execute statemes on closed statements");
     }
     try {
-      statement.executeQuery(s, true, null);
+      statement.executeQuery(s, true, null, new LensConf());
     } catch (Throwable t) {
       throw new SQLException(t);
     }

http://git-wip-us.apache.org/repos/asf/lens/blob/12774400/lens-client/src/test/java/org/apache/lens/client/TestLensClient.java
----------------------------------------------------------------------
diff --git a/lens-client/src/test/java/org/apache/lens/client/TestLensClient.java b/lens-client/src/test/java/org/apache/lens/client/TestLensClient.java
index 94a788a..d9e60fb 100644
--- a/lens-client/src/test/java/org/apache/lens/client/TestLensClient.java
+++ b/lens-client/src/test/java/org/apache/lens/client/TestLensClient.java
@@ -31,6 +31,7 @@ import javax.ws.rs.core.UriBuilder;
 import javax.xml.datatype.DatatypeFactory;
 
 import org.apache.lens.api.APIResult;
+import org.apache.lens.api.LensConf;
 import org.apache.lens.api.metastore.*;
 import org.apache.lens.api.query.LensQuery;
 import org.apache.lens.api.query.QueryHandle;
@@ -202,9 +203,18 @@ public class TestLensClient extends LensAllApplicationJerseyTest {
     for (Map.Entry<String, String> e : queryConf.entrySet()) {
       client.setConnectionParam(e.getKey(), e.getValue());
     }
-    QueryHandle handle = client.executeQueryAsynch(query, "testQuery");
+
+    LensConf conf = new LensConf();
+    Map<String, String> confProps = new HashMap<>();
+    confProps.put("custom.property.for.validation", "present");
+    conf.addProperties(confProps);
+
+    QueryHandle handle = client.executeQueryAsynch(query, "testQuery", conf);
     client.getStatement().waitForQueryToComplete(handle);
     assertTrue(client.getStatement().wasQuerySuccessful());
+    LensQuery persistedQuery = client.getQueryDetails(handle);
+    Assert.assertNotNull(persistedQuery.getQueryConf());
+    Assert.assertEquals(persistedQuery.getQueryConf().getProperty("custom.property.for.validation"), "present");
 
     ResultSet result = null;
     boolean isHeaderRowPresent = columnNamesExpected > 0 ? true : false;
@@ -247,7 +257,7 @@ public class TestLensClient extends LensAllApplicationJerseyTest {
       // Setting very small timeout. Expecting timeouts after this
       // Note: Timeout values can be changed even after LensClient has been created.
       config.setInt(LensClientConfig.READ_TIMEOUT_MILLIS, 200);
-      lensClient.executeQueryWithTimeout("cube select id,name from test_dim", "test1", 100000);
+      lensClient.executeQueryWithTimeout("cube select id,name from test_dim", "test1", 100000, new LensConf());
       fail("Read Timeout was expected");
     } catch (Exception e) {
       if (!(isExceptionDueToSocketTimeout(e))) {
@@ -261,7 +271,7 @@ public class TestLensClient extends LensAllApplicationJerseyTest {
     //Setting back default timeout. Not expecting timeouts after this
     config.setInt(LensClientConfig.READ_TIMEOUT_MILLIS, LensClientConfig.DEFAULT_READ_TIMEOUT_MILLIS);
     QueryHandleWithResultSet result = lensClient.executeQueryWithTimeout("cube select id,name from test_dim", "test2",
-      100000);
+      100000, new LensConf());
     assertTrue(result.getStatus().successful());
     lensClient.closeConnection();
   }
@@ -275,7 +285,8 @@ public class TestLensClient extends LensAllApplicationJerseyTest {
       assertTrue(lensClient.setDatabase(TEST_DB));
 
       //Test waitForQueryToComplete without retry on timeout
-      QueryHandle handle = lensClient.executeQueryAsynch("cube select id,name from test_dim", "test3");
+      QueryHandle handle = lensClient.executeQueryAsynch("cube select id,name from test_dim", "test3",
+        new LensConf());
       try {
         lensClient.getStatement().waitForQueryToComplete(handle, false);
         fail("SocketTimeoutException was expected");
@@ -286,7 +297,7 @@ public class TestLensClient extends LensAllApplicationJerseyTest {
       }
 
       //Test waitForQueryToComplete with Retry on timeout
-      handle = lensClient.executeQueryAsynch("cube select id,name from test_dim", "test3");
+      handle = lensClient.executeQueryAsynch("cube select id,name from test_dim", "test3", new LensConf());
       lensClient.getStatement().waitForQueryToComplete(handle);
       LensQuery query = lensClient.getQueryDetails(handle);
       assertTrue(query.getStatus().successful());

http://git-wip-us.apache.org/repos/asf/lens/blob/12774400/lens-examples/src/main/java/org/apache/lens/examples/SampleQueries.java
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/java/org/apache/lens/examples/SampleQueries.java b/lens-examples/src/main/java/org/apache/lens/examples/SampleQueries.java
index e88ed28..91c048d 100644
--- a/lens-examples/src/main/java/org/apache/lens/examples/SampleQueries.java
+++ b/lens-examples/src/main/java/org/apache/lens/examples/SampleQueries.java
@@ -25,6 +25,7 @@ import java.io.InputStreamReader;
 
 import javax.xml.bind.JAXBException;
 
+import org.apache.lens.api.LensConf;
 import org.apache.lens.api.query.*;
 import org.apache.lens.client.LensClient;
 import org.apache.lens.client.LensClientSingletonWrapper;
@@ -134,7 +135,7 @@ public class SampleQueries {
       total++;
       System.out.println("Query:" + query);
       try {
-        QueryHandle handle = queryClient.executeQuery(query, true, null);
+        QueryHandle handle = queryClient.executeQuery(query, true, null, new LensConf());
         System.out.println("Status:" + queryClient.getQuery().getStatus());
         System.out.println("Total time in millis:"
           + (queryClient.getQuery().getFinishTime() - queryClient.getQuery().getSubmissionTime()));