You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by pu...@apache.org on 2016/05/12 12:55:00 UTC

lens git commit: LENS-1019: Return same handle for repeated query from same user

Repository: lens
Updated Branches:
  refs/heads/master 23ca56644 -> 797899c31


LENS-1019: Return same handle for repeated query from same user


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

Branch: refs/heads/master
Commit: 797899c31aab81947a871d449adf9c00aba8ae3c
Parents: 23ca566
Author: Lavkesh Lahngir <la...@linux.com>
Authored: Thu May 12 18:05:37 2016 +0530
Committer: Puneet <pu...@inmobi.com>
Committed: Thu May 12 18:05:37 2016 +0530

----------------------------------------------------------------------
 .../lens/server/api/LensConfConstants.java      |  12 ++
 .../lens/server/api/query/QueryContext.java     |   4 +
 .../server/query/QueryExecutionServiceImpl.java | 114 +++++++----
 .../lens/server/session/LensSessionImpl.java    |  20 ++
 .../apache/lens/server/util/UtilityMethods.java |  26 +++
 .../src/main/resources/lensserver-default.xml   |   8 +
 .../lens/server/query/TestQueryService.java     | 122 ++++++++++++
 src/site/apt/admin/config.apt                   | 192 ++++++++++---------
 8 files changed, 371 insertions(+), 127 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/797899c3/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 a6df138..85f54d1 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
@@ -151,6 +151,18 @@ public final class LensConfConstants {
   public static final boolean DEFAULT_SERVER_STATE_PERSISTENCE_ENABLED = true;
 
   /**
+   * If this is false and same query is submitted by a user in the same session
+   * and with the same configuration while earlier query is not completed then
+   * lens server will return the handle of the previous query.
+   */
+  public static final String SERVER_DUPLICATE_QUERY_ALLOWED = SERVER_PFX + "duplicate.query.allowed";
+
+  /**
+   * By default same query by same user is not allowed.
+   */
+  public static final boolean DEFAULT_SERVER_DUPLICATE_QUERY_ALLOWED = false;
+
+  /**
    * The Constant SERVER_STATE_PERSIST_LOCATION.
    */
   public static final String SERVER_STATE_PERSIST_LOCATION = SERVER_PFX + "persist.location";

http://git-wip-us.apache.org/repos/asf/lens/blob/797899c3/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryContext.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryContext.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryContext.java
index 2a11f97..379d532 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryContext.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryContext.java
@@ -189,6 +189,10 @@ public class QueryContext extends AbstractQueryContext {
   @Getter
   private transient boolean isDriverResultRegistered;
 
+  @Getter
+  @Setter
+  private byte[] queryConfHash;
+
   transient FailureContext statusUpdateFailures = new FailureContext();
 
   /**

http://git-wip-us.apache.org/repos/asf/lens/blob/797899c3/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
index 97cab86..c9f9b97 100644
--- a/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/query/QueryExecutionServiceImpl.java
@@ -179,6 +179,11 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
   Configuration conf;
 
   /**
+   * Checks if repeated query is allowed
+   */
+  private boolean isDuplicateQueryAllowed;
+
+  /**
    * The query submitter runnable.
    */
   private QuerySubmitter querySubmitterRunnable;
@@ -880,6 +885,10 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
         }
       }
     }
+    // Remove from active queries
+    if (SESSION_MAP.containsKey(ctx.getLensSessionIdentifier())) {
+      getSession(SESSION_MAP.get(ctx.getLensSessionIdentifier())).removeFromActiveQueries(ctx.getQueryHandle());
+    }
     finishedQueries.add(new FinishedQuery(ctx));
     ctx.clearTransientStateAfterLaunch();
   }
@@ -1132,7 +1141,8 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
   public synchronized void init(HiveConf hiveConf) {
     super.init(hiveConf);
     this.conf = hiveConf;
-
+    this.isDuplicateQueryAllowed = conf.getBoolean(LensConfConstants.SERVER_DUPLICATE_QUERY_ALLOWED,
+        LensConfConstants.DEFAULT_SERVER_DUPLICATE_QUERY_ALLOWED);
     try {
       loadQueryComparator();
     } catch (LensException e) {
@@ -1262,13 +1272,16 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
    * @see org.apache.hive.service.CompositeService#start()
    */
   public synchronized void start() {
-    // recover query configurations from session
+    final List<QueryContext> allRestoredQueuedQueries = new LinkedList<QueryContext>();
     synchronized (allQueries) {
       for (QueryContext ctx : allQueries.values()) {
+        // recover query configurations from session
         try {
           if (SESSION_MAP.containsKey(ctx.getLensSessionIdentifier())) {
             // try setting configuration if the query session is still not closed
             ctx.setConf(getLensConf(getSessionHandle(ctx.getLensSessionIdentifier()), ctx.getLensConf()));
+            // Add queryHandle to active queries
+            getSession(SESSION_MAP.get(ctx.getLensSessionIdentifier())).addToActiveQueries(ctx.getQueryHandle());
           } else {
             ctx.setConf(getLensConf(ctx.getLensConf()));
           }
@@ -1280,7 +1293,36 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
         } catch (LensException e) {
           log.error("Could not set query conf ", e);
         }
+        // Add queries to the queue.
+        switch (ctx.getStatus().getStatus()) {
+        case NEW:
+        case QUEUED:
+          allRestoredQueuedQueries.add(ctx);
+          break;
+        case LAUNCHED:
+        case RUNNING:
+        case EXECUTED:
+          try {
+            launchedQueries.add(ctx);
+          } catch (final Exception e) {
+            log.error("Query not restored:QueryContext:{}", ctx, e);
+          }
+          break;
+        case SUCCESSFUL:
+        case FAILED:
+        case CANCELED:
+          updateFinishedQuery(ctx, null);
+          break;
+        case CLOSED:
+          allQueries.remove(ctx.getQueryHandle());
+          if (SESSION_MAP.containsKey(ctx.getLensSessionIdentifier())) {
+            getSession(SESSION_MAP.get(ctx.getLensSessionIdentifier())).removeFromActiveQueries(ctx.getQueryHandle());
+          }
+          log.info("Removed closed query from all Queries:" + ctx.getQueryHandle());
+        }
       }
+      queuedQueries.addAll(allRestoredQueuedQueries);
+      log.info("Recovered {} queries", allQueries.size());
     }
     super.start();
     querySubmitter.start();
@@ -1801,7 +1843,17 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
       acquire(sessionHandle);
       Configuration qconf = getLensConf(sessionHandle, conf);
       accept(query, qconf, SubmitOp.EXECUTE);
+      if (!isDuplicateQueryAllowed) {
+        QueryHandle previousHandle = checkForDuplicateQuery(query, sessionHandle, qconf, queryName);
+        if (previousHandle != null) {
+          log.info("Query:{} Session:{} User:{} duplicate query found", query, sessionHandle, getSession(sessionHandle)
+              .getLoggedInUser());
+          return previousHandle;
+        }
+      }
       QueryContext ctx = createContext(query, getSession(sessionHandle).getLoggedInUser(), conf, qconf, 0);
+      // Should be set only once
+      ctx.setQueryConfHash(UtilityMethods.generateHashOfWritable(qconf));
       ctx.setQueryName(queryName);
       return executeAsyncInternal(sessionHandle, ctx);
     } finally {
@@ -1810,6 +1862,32 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
   }
 
   /**
+   * Returns the query handle if the same query is already launched by the user
+   * in the same session.
+   *
+   * @param query
+   * @param sessionHandle
+   * @param qconf
+   * @param queryName
+   * @return
+   */
+  private QueryHandle checkForDuplicateQuery(String query, LensSessionHandle sessionHandle, Configuration conf,
+      String queryName) {
+    // Get all active queries of this session.
+    List<QueryHandle> activeQueries = getSession(sessionHandle).getActiveQueries();
+    synchronized (activeQueries) {
+      for (QueryHandle handle : activeQueries) {
+        QueryContext context = allQueries.get(handle);
+        if (queryName.equals(context.getQueryName()) && query.equals(context.getUserQuery())
+            && Arrays.equals(UtilityMethods.generateHashOfWritable(conf), context.getQueryConfHash())) {
+          return handle;
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
    * Creates the context.
    *
    * @param query    the query
@@ -1865,6 +1943,8 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
     queuedQueries.add(ctx);
     log.debug("Added to Queued Queries:{}", ctx.getQueryHandleString());
     allQueries.put(ctx.getQueryHandle(), ctx);
+    // Add to session's active query list
+    getSession(SESSION_MAP.get(ctx.getLensSessionIdentifier())).addToActiveQueries(ctx.getQueryHandle());
     fireStatusChangeEvent(ctx, ctx.getStatus(), before);
     log.info("Returning handle {}", ctx.getQueryHandle().getHandleId());
     return ctx.getQueryHandle();
@@ -2634,36 +2714,6 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
         }
         allQueries.put(ctx.getQueryHandle(), ctx);
       }
-
-      // populate the query queues
-      final List<QueryContext> allRestoredQueuedQueries = new LinkedList<QueryContext>();
-      for (QueryContext ctx : allQueries.values()) {
-        switch (ctx.getStatus().getStatus()) {
-        case NEW:
-        case QUEUED:
-          allRestoredQueuedQueries.add(ctx);
-          break;
-        case LAUNCHED:
-        case RUNNING:
-        case EXECUTED:
-          try {
-            launchedQueries.add(ctx);
-          } catch (final Exception e) {
-            log.error("Query not restored:QueryContext:{}", ctx, e);
-          }
-          break;
-        case SUCCESSFUL:
-        case FAILED:
-        case CANCELED:
-          updateFinishedQuery(ctx, null);
-          break;
-        case CLOSED:
-          allQueries.remove(ctx.getQueryHandle());
-          log.info("Removed closed query from all Queries:"+ctx.getQueryHandle());
-        }
-      }
-      queuedQueries.addAll(allRestoredQueuedQueries);
-      log.info("Recovered {} queries", allQueries.size());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/797899c3/lens-server/src/main/java/org/apache/lens/server/session/LensSessionImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/session/LensSessionImpl.java b/lens-server/src/main/java/org/apache/lens/server/session/LensSessionImpl.java
index 900770b..88406c5 100644
--- a/lens-server/src/main/java/org/apache/lens/server/session/LensSessionImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/session/LensSessionImpl.java
@@ -26,6 +26,7 @@ import java.util.*;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lens.api.LensSessionHandle;
+import org.apache.lens.api.query.QueryHandle;
 import org.apache.lens.cube.metadata.CubeMetastoreClient;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.api.LensConfConstants;
@@ -38,6 +39,7 @@ import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.session.SessionState;
+
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.SessionHandle;
 import org.apache.hive.service.cli.session.HiveSessionImpl;
@@ -68,6 +70,12 @@ public class LensSessionImpl extends HiveSessionImpl {
   private Configuration conf = createDefaultConf();
 
   /**
+   * List of queries which are submitted in this session.
+   */
+  @Getter
+  private List<QueryHandle> activeQueries = new ArrayList<QueryHandle>();
+
+  /**
    * Keep track of DB static resources which failed to be added to this session
    */
   private final Map<String, List<ResourceEntry>> failedDBResources = new HashMap<String, List<ResourceEntry>>();
@@ -590,4 +598,16 @@ public class LensSessionImpl extends HiveSessionImpl {
       lastAccessTime = in.readLong();
     }
   }
+
+  public void addToActiveQueries(QueryHandle queryHandle) {
+    synchronized (this.activeQueries) {
+      this.activeQueries.add(queryHandle);
+    }
+  }
+
+  public void removeFromActiveQueries(QueryHandle queryHandle) {
+    synchronized (this.activeQueries) {
+      this.activeQueries.remove(queryHandle);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/797899c3/lens-server/src/main/java/org/apache/lens/server/util/UtilityMethods.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/util/UtilityMethods.java b/lens-server/src/main/java/org/apache/lens/server/util/UtilityMethods.java
index a6c7b13..63a7874 100644
--- a/lens-server/src/main/java/org/apache/lens/server/util/UtilityMethods.java
+++ b/lens-server/src/main/java/org/apache/lens/server/util/UtilityMethods.java
@@ -21,6 +21,8 @@ package org.apache.lens.server.util;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Map;
@@ -34,11 +36,16 @@ import org.apache.commons.dbutils.QueryRunner;
 import org.apache.commons.dbutils.ResultSetHandler;
 import org.apache.commons.pool.impl.GenericObjectPool;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+import lombok.extern.slf4j.Slf4j;
 
 
 /**
  * The Class UtilityMethods.
  */
+@Slf4j
 public final class UtilityMethods {
   private UtilityMethods() {
 
@@ -187,4 +194,23 @@ public final class UtilityMethods {
       os.flush();
     }
   }
+
+  /**
+   * Generates a md5 hash of a writable object.
+   *
+   * @param writable
+   * @return hash of a writable object
+   */
+  public static byte[] generateHashOfWritable(Writable writable) {
+    try {
+      MessageDigest md = MessageDigest.getInstance("MD5");
+      byte [] lensConfBytes = WritableUtils.toByteArray(writable);
+      md.update(lensConfBytes);
+      byte [] digest = md.digest();
+      return digest;
+    } catch (NoSuchAlgorithmException e) {
+      log.warn("MD5: No such method error " + writable);
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/797899c3/lens-server/src/main/resources/lensserver-default.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/main/resources/lensserver-default.xml b/lens-server/src/main/resources/lensserver-default.xml
index 9580397..25e0518 100644
--- a/lens-server/src/main/resources/lensserver-default.xml
+++ b/lens-server/src/main/resources/lensserver-default.xml
@@ -843,6 +843,14 @@
     </description>
   </property>
   <property>
+    <name>lens.server.duplicate.query.allowed</name>
+    <value>false</value>
+    <description>If this is false and same query is submitted by a user in the same session
+      and with the same configuration while earlier query is not completed then
+      lens server will return the handle of the previous query
+    </description>
+  </property>
+  <property>
   <name>lens.server.status.update.exponential.wait.millis</name>
   <value>30000</value>
   <description>Number of millis that would grow exponentially for next update, incase of transient failures.

http://git-wip-us.apache.org/repos/asf/lens/blob/797899c3/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
index 797886d..692a0a0 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
@@ -774,6 +774,128 @@ public class TestQueryService extends LensJerseyTest {
     validateHttpEndPoint(target(), null, handle3, null);
   }
 
+  @Test(dataProvider = "mediaTypeData")
+  public void testExecuteAsyncDuplicate(MediaType mt) throws InterruptedException, IOException, LensException {
+    // test post execute op
+    final WebTarget target = target().path("queryapi/queries");
+
+    queryService.pauseQuerySubmitter(true);
+    final FormDataMultiPart mp = new FormDataMultiPart();
+    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionid").build(), lensSessionId, mt));
+    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("query").build(), "select ID, IDSTR from "
+        + TEST_TABLE));
+    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("operation").build(), "execute"));
+    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
+        mt));
+    // Dummy query
+    final QueryHandle handle = target.request(mt)
+        .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE), new GenericType<LensAPIResult<QueryHandle>>() {
+        }).getData();
+
+    target.path(handle.toString()).queryParam("sessionid", lensSessionId).request(mt).delete(APIResult.class);
+
+    final QueryHandle handle1 = target.request(mt)
+        .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE), new GenericType<LensAPIResult<QueryHandle>>() {
+        }).getData();
+
+    final QueryHandle handle2 = target.request(mt)
+        .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE), new GenericType<LensAPIResult<QueryHandle>>() {
+        }).getData();
+
+    assertNotNull(handle1);
+    assertNotNull(handle2);
+    assertEquals(handle1, handle2);
+    // Cancel the query
+    target.path(handle1.toString()).queryParam("sessionid", lensSessionId).request(mt).delete(APIResult.class);
+
+    // Create a different query
+    final FormDataMultiPart mp1 = new FormDataMultiPart();
+    mp1.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionid").build(), lensSessionId, mt));
+    mp1.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("query").build(), "select ID, IDSTR from "
+        + TEST_TABLE));
+    mp1.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("operation").build(), "execute"));
+    mp1.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
+        mt));
+    final QueryHandle handle3 = target.request(mt)
+        .post(Entity.entity(mp1, MediaType.MULTIPART_FORM_DATA_TYPE), new GenericType<LensAPIResult<QueryHandle>>() {
+        }).getData();
+    assertNotNull(handle3);
+    target.path(handle3.toString()).queryParam("sessionid", lensSessionId).request(mt).delete(APIResult.class);
+
+    // After completion If we launch the same query it should return a new
+    // handle.
+    final QueryHandle handle4 = target.request(mt)
+        .post(Entity.entity(mp1, MediaType.MULTIPART_FORM_DATA_TYPE), new GenericType<LensAPIResult<QueryHandle>>() {
+        }).getData();
+    assertNotEquals(handle4, handle3);
+
+    target.path(handle4.toString()).queryParam("sessionid", lensSessionId).request(mt).delete(APIResult.class);
+    // Launch the query in different session should result in the different
+    // handle.
+
+    final FormDataMultiPart mp2 = new FormDataMultiPart();
+    mp2.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionid").build(), lensSessionId, mt));
+    mp2.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("query").build(), "select ID, IDSTR from "
+        + TEST_TABLE));
+    mp2.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("operation").build(), "execute"));
+    mp2.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
+        mt));
+    final QueryHandle handle5 = target.request(mt)
+        .post(Entity.entity(mp2, MediaType.MULTIPART_FORM_DATA_TYPE), new GenericType<LensAPIResult<QueryHandle>>() {
+        }).getData();
+
+    Map<String, String> sessionconf = new HashMap<>();
+    sessionconf.put("test.session.key", "svalue");
+
+    LensSessionHandle lensSessionId1 = queryService.openSession("foo@localhost", "bar", sessionconf);
+    final FormDataMultiPart mp3 = new FormDataMultiPart();
+    mp3.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionid").build(), lensSessionId1, mt));
+    mp3.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("query").build(), "select ID, IDSTR from "
+        + TEST_TABLE));
+    mp3.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("operation").build(), "execute"));
+    mp3.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
+        mt));
+    final QueryHandle handle6 = target.request(mt)
+        .post(Entity.entity(mp3, MediaType.MULTIPART_FORM_DATA_TYPE), new GenericType<LensAPIResult<QueryHandle>>() {
+        }).getData();
+
+    assertNotNull(handle5);
+    assertNotNull(handle6);
+    assertNotEquals(handle5, handle6);
+    target.path(handle5.toString()).queryParam("sessionid", lensSessionId).request(mt).delete(APIResult.class);
+    target.path(handle6.toString()).queryParam("sessionid", lensSessionId1).request(mt).delete(APIResult.class);
+
+    // Diffrent conf should different handle
+    LensConf conf = new LensConf();
+    final FormDataMultiPart mp4 = new FormDataMultiPart();
+    mp4.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionid").build(), lensSessionId, mt));
+    mp4.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("query").build(), "select ID, IDSTR from "
+        + TEST_TABLE));
+    mp4.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("operation").build(), "execute"));
+    mp4.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf, mt));
+    final QueryHandle handle7 = target.request(mt)
+        .post(Entity.entity(mp4, MediaType.MULTIPART_FORM_DATA_TYPE), new GenericType<LensAPIResult<QueryHandle>>() {
+        }).getData();
+
+    // Add a property
+    conf.addProperty("test", "test");
+    final FormDataMultiPart mp5 = new FormDataMultiPart();
+    mp5.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionid").build(), lensSessionId, mt));
+    mp5.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("query").build(), "select ID, IDSTR from "
+        + TEST_TABLE));
+    mp5.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("operation").build(), "execute"));
+    mp5.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf, mt));
+    final QueryHandle handle8 = target.request(mt)
+        .post(Entity.entity(mp5, MediaType.MULTIPART_FORM_DATA_TYPE), new GenericType<LensAPIResult<QueryHandle>>() {
+        }).getData();
+    assertNotNull(handle7);
+    assertNotNull(handle8);
+    assertNotEquals(handle7, handle8);
+    target.path(handle7.toString()).queryParam("sessionid", lensSessionId).request(mt).delete(APIResult.class);
+    target.path(handle8.toString()).queryParam("sessionid", lensSessionId1).request(mt).delete(APIResult.class);
+    queryService.pauseQuerySubmitter(false);
+  }
+
   /**
    * Validate persisted result.
    *

http://git-wip-us.apache.org/repos/asf/lens/blob/797899c3/src/site/apt/admin/config.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/admin/config.apt b/src/site/apt/admin/config.apt
index 4bf1e7c..40da099 100644
--- a/src/site/apt/admin/config.apt
+++ b/src/site/apt/admin/config.apt
@@ -67,196 +67,198 @@ Lens server configuration
 *--+--+---+--+
 |20|lens.server.drivers|hive:org.apache.lens.driver.hive.HiveDriver|Drivers enabled for this lens server instance|
 *--+--+---+--+
-|21|lens.server.enable.console.metrics|false|Enable metrics to be reported on console|
+|21|lens.server.duplicate.query.allowed|false|If this is false and same query is submitted by a user in the same session and with the same configuration while earlier query is not completed then lens server will return the handle of the previous query|
 *--+--+---+--+
-|22|lens.server.enable.csv.metrics|false|Enable metrics to be reported as csv files in a directory|
+|22|lens.server.enable.console.metrics|false|Enable metrics to be reported on console|
 *--+--+---+--+
-|23|lens.server.enable.ganglia.metrics|false|Enable metrics to be reported on ganglia|
+|23|lens.server.enable.csv.metrics|false|Enable metrics to be reported as csv files in a directory|
 *--+--+---+--+
-|24|lens.server.enable.graphite.metrics|false|Enable metrics to be reported on graphite|
+|24|lens.server.enable.ganglia.metrics|false|Enable metrics to be reported on ganglia|
 *--+--+---+--+
-|25|lens.server.enable.resource.method.metering|false|Whether to Enable metering for all resource methods.|
+|25|lens.server.enable.graphite.metrics|false|Enable metrics to be reported on graphite|
 *--+--+---+--+
-|26|lens.server.estimate.pool.keepalive.millis|60000|Thread keep alive time in milliseconds for the estimate thread pool. If there are no estimate requests for this period,then cached threads are released from the pool.|
+|26|lens.server.enable.resource.method.metering|false|Whether to Enable metering for all resource methods.|
 *--+--+---+--+
-|27|lens.server.estimate.pool.max.threads|100|Maximum number of threads in the estimate thread pool|
+|27|lens.server.estimate.pool.keepalive.millis|60000|Thread keep alive time in milliseconds for the estimate thread pool. If there are no estimate requests for this period,then cached threads are released from the pool.|
 *--+--+---+--+
-|28|lens.server.estimate.pool.min.threads|3|Minimum number of threads in the estimate thread pool|
+|28|lens.server.estimate.pool.max.threads|100|Maximum number of threads in the estimate thread pool|
 *--+--+---+--+
-|29|lens.server.estimate.timeout.millis|300000|Timeout for parallel query estimate calls in milliseconds. A driver needs to comeback with a query estimate within this timeout. If the timeout is reached, only the drivers that have provided an estimate would be considered for query selection. If the timeout is reached and none of the drivers have provided an estimate then estimate calls fails with an exception.|
+|29|lens.server.estimate.pool.min.threads|3|Minimum number of threads in the estimate thread pool|
 *--+--+---+--+
-|30|lens.server.event.service.thread.pool.size| |The size of thread pool for notifying events. The no value is specified, it uses the available processors as the number.|
+|30|lens.server.estimate.timeout.millis|300000|Timeout for parallel query estimate calls in milliseconds. A driver needs to comeback with a query estimate within this timeout. If the timeout is reached, only the drivers that have provided an estimate would be considered for query selection. If the timeout is reached and none of the drivers have provided an estimate then estimate calls fails with an exception.|
 *--+--+---+--+
-|31|lens.server.hdfs.output.retention|1 day|hdfs output retention period. Default 1 day|
+|31|lens.server.event.service.thread.pool.size| |The size of thread pool for notifying events. The no value is specified, it uses the available processors as the number.|
 *--+--+---+--+
-|32|lens.server.index.ws.resource.impl|org.apache.lens.server.IndexResource|Implementation class for Index Resource|
+|32|lens.server.hdfs.output.retention|1 day|hdfs output retention period. Default 1 day|
 *--+--+---+--+
-|33|lens.server.inmemory.resultset.ttl.secs|300|This property defines the TTL(time to live) in seconds for all result sets of type InMemoryResultSet beyond which they are eligible for purging irrespective of whether the result set has been read or not. The default value is 300 seconds (5 minutes).|
+|33|lens.server.index.ws.resource.impl|org.apache.lens.server.IndexResource|Implementation class for Index Resource|
 *--+--+---+--+
-|34|lens.server.log.ws.resource.impl|org.apache.lens.server.LogResource|Implementation class for Log Resource|
+|34|lens.server.inmemory.resultset.ttl.secs|300|This property defines the TTL(time to live) in seconds for all result sets of type InMemoryResultSet beyond which they are eligible for purging irrespective of whether the result set has been read or not. The default value is 300 seconds (5 minutes).|
 *--+--+---+--+
-|35|lens.server.mail.from.address|blah@company.com|The from field in the notifier mail to the submitter.|
+|35|lens.server.log.ws.resource.impl|org.apache.lens.server.LogResource|Implementation class for Log Resource|
 *--+--+---+--+
-|36|lens.server.mail.host|mail-host.company.com|SMTP Host for sending mail|
+|36|lens.server.mail.from.address|blah@company.com|The from field in the notifier mail to the submitter.|
 *--+--+---+--+
-|37|lens.server.mail.port|25|SMTP Port|
+|37|lens.server.mail.host|mail-host.company.com|SMTP Host for sending mail|
 *--+--+---+--+
-|38|lens.server.mail.smtp.connectiontimeout|15000|Socket connection timeout value in milliseconds. This timeout is implemented by java.net.Socket. Default is 15 seconds.|
+|38|lens.server.mail.port|25|SMTP Port|
 *--+--+---+--+
-|39|lens.server.mail.smtp.timeout|30000|Socket read timeout value in milliseconds. This timeout is implemented by java.net.Socket. Default is 30 seconds.|
+|39|lens.server.mail.smtp.connectiontimeout|15000|Socket connection timeout value in milliseconds. This timeout is implemented by java.net.Socket. Default is 15 seconds.|
 *--+--+---+--+
-|40|lens.server.max.sessions.per.user|10|Number of sessions can be allowed for each user. User has to close one of the active sessions to open a new session once limit is reached. Otherwise Server throws an exception by saying that opened session limit has been already reached for user.|
+|40|lens.server.mail.smtp.timeout|30000|Socket read timeout value in milliseconds. This timeout is implemented by java.net.Socket. Default is 30 seconds.|
 *--+--+---+--+
-|41|lens.server.metastore.service.impl|org.apache.lens.server.metastore.CubeMetastoreServiceImpl|Implementation class for metastore service|
+|41|lens.server.max.sessions.per.user|10|Number of sessions can be allowed for each user. User has to close one of the active sessions to open a new session once limit is reached. Otherwise Server throws an exception by saying that opened session limit has been already reached for user.|
 *--+--+---+--+
-|42|lens.server.metastore.ws.resource.impl|org.apache.lens.server.metastore.MetastoreResource|Implementation class for Metastore Resource|
+|42|lens.server.metastore.service.impl|org.apache.lens.server.metastore.CubeMetastoreServiceImpl|Implementation class for metastore service|
 *--+--+---+--+
-|43|lens.server.metrics.csv.directory.path|metrics/|Path of the directory in which to report metrics as separate csv files.|
+|43|lens.server.metastore.ws.resource.impl|org.apache.lens.server.metastore.MetastoreResource|Implementation class for Metastore Resource|
 *--+--+---+--+
-|44|lens.server.metrics.ganglia.host| |The ganglia host name|
+|44|lens.server.metrics.csv.directory.path|metrics/|Path of the directory in which to report metrics as separate csv files.|
 *--+--+---+--+
-|45|lens.server.metrics.ganglia.port| |The ganglia port|
+|45|lens.server.metrics.ganglia.host| |The ganglia host name|
 *--+--+---+--+
-|46|lens.server.metrics.graphite.host| |The graphite host name|
+|46|lens.server.metrics.ganglia.port| |The ganglia port|
 *--+--+---+--+
-|47|lens.server.metrics.graphite.port| |The graphite port|
+|47|lens.server.metrics.graphite.host| |The graphite host name|
 *--+--+---+--+
-|48|lens.server.metrics.reporting.period|10|The reporting period for metrics. The value is in seconds|
+|48|lens.server.metrics.graphite.port| |The graphite port|
 *--+--+---+--+
-|49|lens.server.mode|OPEN|The mode in which server should run. Allowed values are OPEN, READ_ONLY, METASTORE_READONLY, METASTORE_NODROP. OPEN mode will allow all requests. READ_ONLY mode will allow all requests on session resouce and only GET requests on all other resources. METASTORE_READONLY will allow GET on metastore and all other requests in other services. METASTORE_NODROP will not allow DELETE on metastore, will allow all other requests.|
+|49|lens.server.metrics.reporting.period|10|The reporting period for metrics. The value is in seconds|
 *--+--+---+--+
-|50|lens.server.moxyjson.ws.feature.impl|org.glassfish.jersey.moxy.json.MoxyJsonFeature|Enable Moxy json feature|
+|50|lens.server.mode|OPEN|The mode in which server should run. Allowed values are OPEN, READ_ONLY, METASTORE_READONLY, METASTORE_NODROP. OPEN mode will allow all requests. READ_ONLY mode will allow all requests on session resouce and only GET requests on all other resources. METASTORE_READONLY will allow GET on metastore and all other requests in other services. METASTORE_NODROP will not allow DELETE on metastore, will allow all other requests.|
 *--+--+---+--+
-|51|lens.server.moxyjsonconfigresovler.ws.feature.impl|org.apache.lens.api.util.MoxyJsonConfigurationContextResolver|Moxy json configuration resolver|
+|51|lens.server.moxyjson.ws.feature.impl|org.glassfish.jersey.moxy.json.MoxyJsonFeature|Enable Moxy json feature|
 *--+--+---+--+
-|52|lens.server.multipart.ws.feature.impl|org.glassfish.jersey.media.multipart.MultiPartFeature|Implementation class for query scheduler resource|
+|52|lens.server.moxyjsonconfigresovler.ws.feature.impl|org.apache.lens.api.util.MoxyJsonConfigurationContextResolver|Moxy json configuration resolver|
 *--+--+---+--+
-|53|lens.server.persist.location|file:///tmp/lensserver|The directory in which lens server will persist its state when it is going down. The location be on any Hadoop compatible file system. Server will read from the location when it is restarted and recovery is enabled. So, Server should have both read and write permissions to the location|
+|53|lens.server.multipart.ws.feature.impl|org.glassfish.jersey.media.multipart.MultiPartFeature|Implementation class for query scheduler resource|
 *--+--+---+--+
-|54|lens.server.query.acceptors| |Query Acceptors configured. Query acceptors are consulted first, before anything happens for the given query. They can either return null or return a messaging indicating why the given query shouldn't be accepted. These can be used to filter out queries at the earliest.|
+|54|lens.server.persist.location|file:///tmp/lensserver|The directory in which lens server will persist its state when it is going down. The location be on any Hadoop compatible file system. Server will read from the location when it is restarted and recovery is enabled. So, Server should have both read and write permissions to the location|
 *--+--+---+--+
-|55|lens.server.query.launching.constraint.factories|org.apache.lens.server.query.constraint.TotalQueryCostCeilingConstraintFactory|Factories used to instantiate constraints enforced on queries by lens. Every Factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.constraint.QueryLaunchingConstraint. A query will be launched only if all constraints pass.|
+|55|lens.server.query.acceptors| |Query Acceptors configured. Query acceptors are consulted first, before anything happens for the given query. They can either return null or return a messaging indicating why the given query shouldn't be accepted. These can be used to filter out queries at the earliest.|
 *--+--+---+--+
-|56|lens.server.query.phase1.rewriters| |Query phase 1 rewriters. This is to convert user query to cube query. The resulting cube query will be passed for validation and rewriting to hql query.\ |
+|56|lens.server.query.launching.constraint.factories|org.apache.lens.server.query.constraint.TotalQueryCostCeilingConstraintFactory|Factories used to instantiate constraints enforced on queries by lens. Every Factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.constraint.QueryLaunchingConstraint. A query will be launched only if all constraints pass.|
+*--+--+---+--+
+|57|lens.server.query.phase1.rewriters| |Query phase 1 rewriters. This is to convert user query to cube query. The resulting cube query will be passed for validation and rewriting to hql query.\ |
 |  |                                  | |Use cases will be to use extra intelligence to convert user query to optimized cube query.                                                              \ |
 |  |                                  | |Or define shortcuts for certain frequently used queries :)                                                                                                |
 *--+--+---+--+
-|57|lens.server.query.resultset.retention|1 day|Lens query resultset retention period. Default 1 day|
+|58|lens.server.query.resultset.retention|1 day|Lens query resultset retention period. Default 1 day|
 *--+--+---+--+
-|58|lens.server.query.service.impl|org.apache.lens.server.query.QueryExecutionServiceImpl|Implementation class for query execution service|
+|59|lens.server.query.service.impl|org.apache.lens.server.query.QueryExecutionServiceImpl|Implementation class for query execution service|
 *--+--+---+--+
-|59|lens.server.query.state.logger.enabled|true|Disable or enable the query state logger with this config. The location for the logger can be specified in logback xml for the class org.apache.lens.server.query.QueryExecutionServiceImpl.QueryStatusLogger|
+|60|lens.server.query.state.logger.enabled|true|Disable or enable the query state logger with this config. The location for the logger can be specified in logback xml for the class org.apache.lens.server.query.QueryExecutionServiceImpl.QueryStatusLogger|
 *--+--+---+--+
-|60|lens.server.query.ws.resource.impl|org.apache.lens.server.query.QueryServiceResource|Implementation class for Query Resource|
+|61|lens.server.query.ws.resource.impl|org.apache.lens.server.query.QueryServiceResource|Implementation class for Query Resource|
 *--+--+---+--+
-|61|lens.server.querypurger.sleep.interval|10000|The interval(milliseconds) with which purger to run periodically. Default 10 sec.|
+|62|lens.server.querypurger.sleep.interval|10000|The interval(milliseconds) with which purger to run periodically. Default 10 sec.|
 *--+--+---+--+
-|62|lens.server.quota.service.impl|org.apache.lens.server.quota.QuotaServiceImpl|Implementation class for quota service|
+|63|lens.server.quota.service.impl|org.apache.lens.server.quota.QuotaServiceImpl|Implementation class for quota service|
 *--+--+---+--+
-|63|lens.server.quota.ws.resource.impl|org.apache.lens.server.quota.QuotaResource|Implementation class for Quota Resource|
+|64|lens.server.quota.ws.resource.impl|org.apache.lens.server.quota.QuotaResource|Implementation class for Quota Resource|
 *--+--+---+--+
-|64|lens.server.resultset.purge.enabled|false|Whether to purge the query results|
+|65|lens.server.resultset.purge.enabled|false|Whether to purge the query results|
 *--+--+---+--+
-|65|lens.server.resultsetpurger.sleep.interval.secs|3600|Periodicity for Query result purger runs. Default 1 hour.|
+|66|lens.server.resultsetpurger.sleep.interval.secs|3600|Periodicity for Query result purger runs. Default 1 hour.|
 *--+--+---+--+
-|66|lens.server.savedquery.jdbc.dialectclass|org.apache.lens.server.query.save.SavedQueryDao$HSQLDialect|Dialect of the target DB, Default is HSQL. Override with the target DB used.|
+|67|lens.server.savedquery.jdbc.dialectclass|org.apache.lens.server.query.save.SavedQueryDao$HSQLDialect|Dialect of the target DB, Default is HSQL. Override with the target DB used.|
 *--+--+---+--+
-|67|lens.server.savedquery.list.default.count|20|Key denoting the default fetch value of saved query list api.|
+|68|lens.server.savedquery.list.default.count|20|Key denoting the default fetch value of saved query list api.|
 *--+--+---+--+
-|68|lens.server.savedquery.list.default.offset|0|Key denoting the default start value of saved query list api.|
+|69|lens.server.savedquery.list.default.offset|0|Key denoting the default start value of saved query list api.|
 *--+--+---+--+
-|69|lens.server.savedquery.service.impl|org.apache.lens.server.query.save.SavedQueryServiceImpl|Implementation class for saved query service|
+|70|lens.server.savedquery.service.impl|org.apache.lens.server.query.save.SavedQueryServiceImpl|Implementation class for saved query service|
 *--+--+---+--+
-|70|lens.server.savedquery.ws.resource.impl|org.apache.lens.server.query.save.SavedQueryResource|Implementation class for Saved query Resource|
+|71|lens.server.savedquery.ws.resource.impl|org.apache.lens.server.query.save.SavedQueryResource|Implementation class for Saved query Resource|
 *--+--+---+--+
-|71|lens.server.scheduler.service.impl|org.apache.lens.server.scheduler.SchedulerServiceImpl|Implementation class for query scheduler service|
+|72|lens.server.scheduler.service.impl|org.apache.lens.server.scheduler.SchedulerServiceImpl|Implementation class for query scheduler service|
 *--+--+---+--+
-|72|lens.server.scheduler.ws.resource.impl|org.apache.lens.server.scheduler.ScheduleResource|Implementation class for query scheduler resource|
+|73|lens.server.scheduler.ws.resource.impl|org.apache.lens.server.scheduler.ScheduleResource|Implementation class for query scheduler resource|
 *--+--+---+--+
-|73|lens.server.scheduling.queue.poll.interval.millisec|2000|The interval at which submission thread will poll scheduling queue to fetch the next query for submission. If value is less than equal to 0, then it would mean that thread will continuosly poll without sleeping. The interval has to be given in milliseconds.|
+|74|lens.server.scheduling.queue.poll.interval.millisec|2000|The interval at which submission thread will poll scheduling queue to fetch the next query for submission. If value is less than equal to 0, then it would mean that thread will continuosly poll without sleeping. The interval has to be given in milliseconds.|
 *--+--+---+--+
-|74|lens.server.serverMode.ws.filter.impl|org.apache.lens.server.ServerModeFilter|Implementation class for ServerMode Filter|
+|75|lens.server.serverMode.ws.filter.impl|org.apache.lens.server.ServerModeFilter|Implementation class for ServerMode Filter|
 *--+--+---+--+
-|75|lens.server.service.provider.factory|org.apache.lens.server.ServiceProviderFactoryImpl|Service provider factory implementation class. This parameter is used to lookup the factory implementation class name that would provide an instance of ServiceProvider. Users should instantiate the class to obtain its instance. Example -- Class spfClass = conf.getClass("lens.server.service.provider.factory", null, ServiceProviderFactory.class); ServiceProviderFactory spf = spfClass.newInstance(); ServiceProvider serviceProvider = spf.getServiceProvider(); -- This is not supposed to be overridden by users.|
+|76|lens.server.service.provider.factory|org.apache.lens.server.ServiceProviderFactoryImpl|Service provider factory implementation class. This parameter is used to lookup the factory implementation class name that would provide an instance of ServiceProvider. Users should instantiate the class to obtain its instance. Example -- Class spfClass = conf.getClass("lens.server.service.provider.factory", null, ServiceProviderFactory.class); ServiceProviderFactory spf = spfClass.newInstance(); ServiceProvider serviceProvider = spf.getServiceProvider(); -- This is not supposed to be overridden by users.|
 *--+--+---+--+
-|76|lens.server.servicenames|session,query,metastore,scheduler,quota|These services would be started in the specified order when lens-server starts up|
+|77|lens.server.servicenames|session,query,metastore,scheduler,quota|These services would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|77|lens.server.session.expiry.service.interval.secs|3600|Interval at which lens session expiry service runs|
+|78|lens.server.session.expiry.service.interval.secs|3600|Interval at which lens session expiry service runs|
 *--+--+---+--+
-|78|lens.server.session.service.impl|org.apache.lens.server.session.HiveSessionService|Implementation class for session service|
+|79|lens.server.session.service.impl|org.apache.lens.server.session.HiveSessionService|Implementation class for session service|
 *--+--+---+--+
-|79|lens.server.session.timeout.seconds|86400|Lens session timeout in seconds.If there is no activity on the session for this period then the session will be closed.Default timeout is one day.|
+|80|lens.server.session.timeout.seconds|86400|Lens session timeout in seconds.If there is no activity on the session for this period then the session will be closed.Default timeout is one day.|
 *--+--+---+--+
-|80|lens.server.session.ws.resource.impl|org.apache.lens.server.session.SessionResource|Implementation class for Session Resource|
+|81|lens.server.session.ws.resource.impl|org.apache.lens.server.session.SessionResource|Implementation class for Session Resource|
 *--+--+---+--+
-|81|lens.server.state.persist.out.stream.buffer.size|1048576|Output Stream Buffer Size used in writing lens server state to file system. Size is in bytes.|
+|82|lens.server.state.persist.out.stream.buffer.size|1048576|Output Stream Buffer Size used in writing lens server state to file system. Size is in bytes.|
 *--+--+---+--+
-|82|lens.server.state.persistence.enabled|true|If flag is enabled, state of all the services will be persisted periodically to a location specified by lens.server.persist.location and on server restart all the services will be started from last saved state.|
+|83|lens.server.state.persistence.enabled|true|If flag is enabled, state of all the services will be persisted periodically to a location specified by lens.server.persist.location and on server restart all the services will be started from last saved state.|
 *--+--+---+--+
-|83|lens.server.state.persistence.interval.millis|300000|Lens server state persistence time interval in milliseconds|
+|84|lens.server.state.persistence.interval.millis|300000|Lens server state persistence time interval in milliseconds|
 *--+--+---+--+
-|84|lens.server.statistics.db|lensstats|Database to which statistics tables are created and partitions are added.|
+|85|lens.server.statistics.db|lensstats|Database to which statistics tables are created and partitions are added.|
 *--+--+---+--+
-|85|lens.server.statistics.log.rollover.interval|3600000|Default rate which log statistics store scans for rollups in milliseconds.|
+|86|lens.server.statistics.log.rollover.interval|3600000|Default rate which log statistics store scans for rollups in milliseconds.|
 *--+--+---+--+
-|86|lens.server.statistics.store.class|org.apache.lens.server.stats.store.log.LogStatisticsStore|Default implementation of class used to persist Lens Statistics.|
+|87|lens.server.statistics.store.class|org.apache.lens.server.stats.store.log.LogStatisticsStore|Default implementation of class used to persist Lens Statistics.|
 *--+--+---+--+
-|87|lens.server.statistics.warehouse.dir|file:///tmp/lens/statistics/warehouse|Default top level location where stats are moved by the log statistics store.|
+|88|lens.server.statistics.warehouse.dir|file:///tmp/lens/statistics/warehouse|Default top level location where stats are moved by the log statistics store.|
 *--+--+---+--+
-|88|lens.server.status.update.exponential.wait.millis|30000|Number of millis that would grow exponentially for next update, incase of transient failures.|
+|89|lens.server.status.update.exponential.wait.millis|30000|Number of millis that would grow exponentially for next update, incase of transient failures.|
 *--+--+---+--+
-|89|lens.server.status.update.maximum.delay.secs|1800|The maximum delay in seconds for next status update to happen after any transient failure. This will be used a maximum delay sothat exponential wait times not to grow to bigger value.|
+|90|lens.server.status.update.maximum.delay.secs|1800|The maximum delay in seconds for next status update to happen after any transient failure. This will be used a maximum delay sothat exponential wait times not to grow to bigger value.|
 *--+--+---+--+
-|90|lens.server.status.update.num.retries|10|The number of retries a status update will tried with exponentital back off, in case of transient issues, upon which query will be marked FAILED.|
+|91|lens.server.status.update.num.retries|10|The number of retries a status update will tried with exponentital back off, in case of transient issues, upon which query will be marked FAILED.|
 *--+--+---+--+
-|91|lens.server.total.query.cost.ceiling.per.user|-1.0|A query submitted by user will be launched only if total query cost of all current launched queries of user is less than or equal to total query cost ceiling defined by this property. This configuration value is only useful when TotalQueryCostCeilingConstraint is enabled by using org.apache.lens.server.query.constraint.TotalQueryCostCeilingConstraintFactory as one of the factories in lens.server.query.constraint.factories property. Default is -1.0 which means that there is no limit on the total query cost of launched queries submitted by a user.|
+|92|lens.server.total.query.cost.ceiling.per.user|-1.0|A query submitted by user will be launched only if total query cost of all current launched queries of user is less than or equal to total query cost ceiling defined by this property. This configuration value is only useful when TotalQueryCostCeilingConstraint is enabled by using org.apache.lens.server.query.constraint.TotalQueryCostCeilingConstraintFactory as one of the factories in lens.server.query.constraint.factories property. Default is -1.0 which means that there is no limit on the total query cost of launched queries submitted by a user.|
 *--+--+---+--+
-|92|lens.server.ui.base.uri|http://0.0.0.0:19999/|The base url for the Lens UI Server|
+|93|lens.server.ui.base.uri|http://0.0.0.0:19999/|The base url for the Lens UI Server|
 *--+--+---+--+
-|93|lens.server.ui.enable|true|Bringing up the ui server is optional. By default it brings up UI server.|
+|94|lens.server.ui.enable|true|Bringing up the ui server is optional. By default it brings up UI server.|
 *--+--+---+--+
-|94|lens.server.ui.enable.caching|true|Set this to false to disable static file caching in the UI server|
+|95|lens.server.ui.enable.caching|true|Set this to false to disable static file caching in the UI server|
 *--+--+---+--+
-|95|lens.server.ui.static.dir|webapp/lens-server/static|The base directory to server UI static files from|
+|96|lens.server.ui.static.dir|webapp/lens-server/static|The base directory to server UI static files from|
 *--+--+---+--+
-|96|lens.server.user.resolver.custom.class|full.package.name.Classname|Required for CUSTOM user resolver. In case the provided implementations are not sufficient for user config resolver, a custom classname can be provided. Class should extend org.apache.lens.server.user.UserConfigLoader|
+|97|lens.server.user.resolver.custom.class|full.package.name.Classname|Required for CUSTOM user resolver. In case the provided implementations are not sufficient for user config resolver, a custom classname can be provided. Class should extend org.apache.lens.server.user.UserConfigLoader|
 *--+--+---+--+
-|97|lens.server.user.resolver.db.keys|lens.session.cluster.user,mapred.job.queue.name|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loaders, the conf keys that will be loaded from database.|
+|98|lens.server.user.resolver.db.keys|lens.session.cluster.user,mapred.job.queue.name|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loaders, the conf keys that will be loaded from database.|
 *--+--+---+--+
-|98|lens.server.user.resolver.db.query|select clusteruser,queue from user_config_table where username=?|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loader, this query will be run with single argument = logged in user and the result columns will be assigned to lens.server.user.resolver.db.keys in order. For ldap backed database resolver, the argument to this query will be the intermediate values obtained from ldap.|
+|99|lens.server.user.resolver.db.query|select clusteruser,queue from user_config_table where username=?|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loader, this query will be run with single argument = logged in user and the result columns will be assigned to lens.server.user.resolver.db.keys in order. For ldap backed database resolver, the argument to this query will be the intermediate values obtained from ldap.|
 *--+--+---+--+
-|99|lens.server.user.resolver.fixed.value| |Required for FIXED user resolver. when lens.server.user.resolver.type=FIXED, This will be the value cluster user will resolve to.|
+|100|lens.server.user.resolver.fixed.value| |Required for FIXED user resolver. when lens.server.user.resolver.type=FIXED, This will be the value cluster user will resolve to.|
 *--+--+---+--+
-|100|lens.server.user.resolver.ldap.bind.dn| |Required for LDAP_BACKED_DATABASE user resolvers. ldap dn for admin binding example: CN=company-it-admin,ou=service-account,ou=company-service-account,dc=dc1,dc=com...|
+|101|lens.server.user.resolver.ldap.bind.dn| |Required for LDAP_BACKED_DATABASE user resolvers. ldap dn for admin binding example: CN=company-it-admin,ou=service-account,ou=company-service-account,dc=dc1,dc=com...|
 *--+--+---+--+
-|101|lens.server.user.resolver.ldap.bind.password| |Required for LDAP_BACKED_DATABASE user resolvers. ldap password for admin binding above|
+|102|lens.server.user.resolver.ldap.bind.password| |Required for LDAP_BACKED_DATABASE user resolvers. ldap password for admin binding above|
 *--+--+---+--+
-|102|lens.server.user.resolver.ldap.fields|department|Required for LDAP_BACKED_DATABASE user resolvers. list of fields to be obtained from ldap. These will be cached by the intermediate db.|
+|103|lens.server.user.resolver.ldap.fields|department|Required for LDAP_BACKED_DATABASE user resolvers. list of fields to be obtained from ldap. These will be cached by the intermediate db.|
 *--+--+---+--+
-|103|lens.server.user.resolver.ldap.intermediate.db.delete.sql|delete from user_department where username=?|Required for LDAP_BACKED_DATABASE user resolvers. query to delete intermediate values from database backing ldap as cache. one argument: logged in user.|
+|104|lens.server.user.resolver.ldap.intermediate.db.delete.sql|delete from user_department where username=?|Required for LDAP_BACKED_DATABASE user resolvers. query to delete intermediate values from database backing ldap as cache. one argument: logged in user.|
 *--+--+---+--+
-|104|lens.server.user.resolver.ldap.intermediate.db.insert.sql|insert into user_department (username, department, expiry) values (?, ?, ?)|Required for LDAP_BACKED_DATABASE user resolvers. query to insert intermediate values from database backing ldap as cache. arguments: first logged in user, then all intermediate values, then current time + expiration time|
+|105|lens.server.user.resolver.ldap.intermediate.db.insert.sql|insert into user_department (username, department, expiry) values (?, ?, ?)|Required for LDAP_BACKED_DATABASE user resolvers. query to insert intermediate values from database backing ldap as cache. arguments: first logged in user, then all intermediate values, then current time + expiration time|
 *--+--+---+--+
-|105|lens.server.user.resolver.ldap.intermediate.db.query|select department from user_department where username=? and expiry>?|Required for LDAP_BACKED_DATABASE user resolvers. query to obtain intermediate values from database backing ldap as cache. two arguments: logged in user and current time.|
+|106|lens.server.user.resolver.ldap.intermediate.db.query|select department from user_department where username=? and expiry>?|Required for LDAP_BACKED_DATABASE user resolvers. query to obtain intermediate values from database backing ldap as cache. two arguments: logged in user and current time.|
 *--+--+---+--+
-|106|lens.server.user.resolver.ldap.search.base| |Required for LDAP_BACKED_DATABASE user resolvers. for searching intermediate values for a user, the search keys. example: cn=users,dc=dc1,dc=dc2...|
+|107|lens.server.user.resolver.ldap.search.base| |Required for LDAP_BACKED_DATABASE user resolvers. for searching intermediate values for a user, the search keys. example: cn=users,dc=dc1,dc=dc2...|
 *--+--+---+--+
-|107|lens.server.user.resolver.ldap.search.filter|(&(objectClass=user)(sAMAccountName=%s))|Required for LDAP_BACKED_DATABASE user resolvers. filter pattern for ldap search|
+|108|lens.server.user.resolver.ldap.search.filter|(&(objectClass=user)(sAMAccountName=%s))|Required for LDAP_BACKED_DATABASE user resolvers. filter pattern for ldap search|
 *--+--+---+--+
-|108|lens.server.user.resolver.ldap.url| |Required for LDAP_BACKED_DATABASE user resolvers. ldap url to connect to.|
+|109|lens.server.user.resolver.ldap.url| |Required for LDAP_BACKED_DATABASE user resolvers. ldap url to connect to.|
 *--+--+---+--+
-|109|lens.server.user.resolver.propertybased.filename|/path/to/propertyfile|Required for PROPERTYBASED user resolver. when lens.server.user.resolver.type is PROPERTYBASED, then this file will be read and parsed to determine cluster user. Each line should contain username followed by DOT followed by property full name followed by equal-to sign and followed by value. example schema of the file is: user1.lens.server.cluster.user=clusteruser1 user1.mapred.job.queue.name=queue1 *.lens.server.cluster.user=defaultclusteruser *.mapred.job.queue.name=default|
+|110|lens.server.user.resolver.propertybased.filename|/path/to/propertyfile|Required for PROPERTYBASED user resolver. when lens.server.user.resolver.type is PROPERTYBASED, then this file will be read and parsed to determine cluster user. Each line should contain username followed by DOT followed by property full name followed by equal-to sign and followed by value. example schema of the file is: user1.lens.server.cluster.user=clusteruser1 user1.mapred.job.queue.name=queue1 *.lens.server.cluster.user=defaultclusteruser *.mapred.job.queue.name=default|
 *--+--+---+--+
-|110|lens.server.user.resolver.type|FIXED|Type of user config resolver. allowed values are FIXED, PROPERTYBASED, DATABASE, LDAP_BACKED_DATABASE, CUSTOM.|
+|111|lens.server.user.resolver.type|FIXED|Type of user config resolver. allowed values are FIXED, PROPERTYBASED, DATABASE, LDAP_BACKED_DATABASE, CUSTOM.|
 *--+--+---+--+
-|111|lens.server.waiting.queries.selection.policy.factories|org.apache.lens.server.query.collect.UserSpecificWaitingQueriesSelectionPolicyFactory|Factories used to instantiate waiting queries selection policies. Every factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.|
+|112|lens.server.waiting.queries.selection.policy.factories|org.apache.lens.server.query.collect.UserSpecificWaitingQueriesSelectionPolicyFactory|Factories used to instantiate waiting queries selection policies. Every factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.|
 *--+--+---+--+
-|112|lens.server.ws.featurenames|multipart,moxyjson,moxyjsonconfigresovler|These JAX-RS Feature(s) would be started in the specified order when lens-server starts up|
+|113|lens.server.ws.featurenames|multipart,moxyjson,moxyjsonconfigresovler|These JAX-RS Feature(s) would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|113|lens.server.ws.filternames|authentication,consistentState,serverMode|These JAX-RS filters would be started in the specified order when lens-server starts up|
+|114|lens.server.ws.filternames|authentication,consistentState,serverMode|These JAX-RS filters would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|114|lens.server.ws.listenernames|appevent|These listeners would be called in the specified order when lens-server starts up|
+|115|lens.server.ws.listenernames|appevent|These listeners would be called in the specified order when lens-server starts up|
 *--+--+---+--+
-|115|lens.server.ws.resourcenames|session,metastore,query,quota,scheduler,index,log|These JAX-RS resources would be started in the specified order when lens-server starts up|
+|116|lens.server.ws.resourcenames|session,metastore,query,quota,scheduler,index,log|These JAX-RS resources would be started in the specified order when lens-server starts up|
 *--+--+---+--+
 The configuration parameters and their default values