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 2017/03/31 09:48:09 UTC

[01/16] lens git commit: LENS-1379 : Fix session expiry for sessions in which operations were done

Repository: lens
Updated Branches:
  refs/heads/lens-1381 d45c5384c -> 8868b0638


LENS-1379 : Fix session expiry for sessions in which operations were done


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

Branch: refs/heads/lens-1381
Commit: 98990c39f4f4826beaf59afb0ef9961f566000c3
Parents: fe66131
Author: Amareshwari Sriramadasu <am...@gmail.com>
Authored: Mon Jan 2 17:45:24 2017 +0530
Committer: Puneet <pu...@inmobi.com>
Committed: Mon Jan 2 17:45:24 2017 +0530

----------------------------------------------------------------------
 .../lens/server/session/HiveSessionService.java | 58 ++++++++--------
 .../lens/server/session/LensSessionImpl.java    | 20 +++---
 .../TestQueryIndependenceFromSessionClose.java  | 71 +++++++++++++++++---
 3 files changed, 102 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/98990c39/lens-server/src/main/java/org/apache/lens/server/session/HiveSessionService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/session/HiveSessionService.java b/lens-server/src/main/java/org/apache/lens/server/session/HiveSessionService.java
index 21e2a62..b480d14 100644
--- a/lens-server/src/main/java/org/apache/lens/server/session/HiveSessionService.java
+++ b/lens-server/src/main/java/org/apache/lens/server/session/HiveSessionService.java
@@ -276,17 +276,19 @@ public class HiveSessionService extends BaseLensService implements SessionServic
    */
   @Override
   public void setSessionParameter(LensSessionHandle sessionid, String key, String value) {
-    setSessionParameter(sessionid, key, value, true);
+    HashMap<String, String> config = Maps.newHashMap();
+    config.put(key, value);
+    setSessionParameters(sessionid, config);
   }
+
   /**
    * Sets the session parameter.
    *
    * @param sessionid    the sessionid
    * @param config       map of string-string. each entry represents key and the value to be set for that key
-   * @param addToSession the add to session
    */
 
-  protected void setSessionParameters(LensSessionHandle sessionid, Map<String, String> config, boolean addToSession) {
+  protected void setSessionParameters(LensSessionHandle sessionid, Map<String, String> config) {
     log.info("Request to Set params:" + config);
     try {
       acquire(sessionid);
@@ -297,17 +299,11 @@ public class HiveSessionService extends BaseLensService implements SessionServic
           var = var.substring(SystemVariables.HIVECONF_PREFIX.length());
         }
         getSession(sessionid).getSessionConf().set(var, entry.getValue());
-        if (addToSession) {
-          String command = "set" + " " + entry.getKey() + "= " + entry.getValue();
-          closeCliServiceOp(getCliService().executeStatement(getHiveSessionHandle(sessionid), command, null));
-        } else {
-          getSession(sessionid).getHiveConf().set(entry.getKey(), entry.getValue());
-        }
+        String command = "set" + " " + entry.getKey() + "= " + entry.getValue();
+        closeCliServiceOp(getCliService().executeStatement(getHiveSessionHandle(sessionid), command, null));
       }
       // add to persist
-      if (addToSession) {
-        getSession(sessionid).setConfig(config);
-      }
+      getSession(sessionid).setConfig(config);
       log.info("Set params:" + config);
     } catch (HiveSQLException e) {
       throw new WebApplicationException(e);
@@ -315,18 +311,18 @@ public class HiveSessionService extends BaseLensService implements SessionServic
       release(sessionid);
     }
   }
-    /**
-     * Sets the session parameter.
-     *
-     * @param sessionid    the sessionid
-     * @param key          the key
-     * @param value        the value
-     * @param addToSession the add to session
-     */
-  protected void setSessionParameter(LensSessionHandle sessionid, String key, String value, boolean addToSession) {
-    HashMap<String, String> config = Maps.newHashMap();
-    config.put(key, value);
-    setSessionParameters(sessionid, config, addToSession);
+
+  private void setSessionParametersOnRestore(LensSessionHandle sessionid, Map<String, String> config) {
+    // set in session conf
+    for(Map.Entry<String, String> entry: config.entrySet()) {
+      String var = entry.getKey();
+      if (var.indexOf(SystemVariables.HIVECONF_PREFIX) == 0) {
+        var = var.substring(SystemVariables.HIVECONF_PREFIX.length());
+      }
+      getSession(sessionid).getSessionConf().set(var, entry.getValue());
+      getSession(sessionid).getHiveConf().set(entry.getKey(), entry.getValue());
+    }
+    log.info("Set params on restart:" + config);
   }
 
   /*
@@ -367,7 +363,7 @@ public class HiveSessionService extends BaseLensService implements SessionServic
         LensSessionHandle sessionHandle = persistInfo.getSessionHandle();
         restoreSession(sessionHandle, persistInfo.getUsername(), persistInfo.getPassword());
         LensSessionImpl session = getSession(sessionHandle);
-        session.setLastAccessTime(persistInfo.getLastAccessTime());
+        session.getLensSessionPersistInfo().setLastAccessTime(persistInfo.getLastAccessTime());
         session.getLensSessionPersistInfo().setConfig(persistInfo.getConfig());
         session.getLensSessionPersistInfo().setResources(persistInfo.getResources());
         session.setCurrentDatabase(persistInfo.getDatabase());
@@ -384,7 +380,7 @@ public class HiveSessionService extends BaseLensService implements SessionServic
 
         // Add config for restored sessions
         try{
-          setSessionParameters(sessionHandle, session.getConfig(), false);
+          setSessionParametersOnRestore(sessionHandle, session.getConfig());
         } catch (Exception e) {
           log.error("Error setting parameters " + session.getConfig()
             + " for session: " + session, e);
@@ -504,7 +500,7 @@ public class HiveSessionService extends BaseLensService implements SessionServic
     }
   }
 
-  Runnable getSessionExpiryRunnable() {
+  public Runnable getSessionExpiryRunnable() {
     return sessionExpiryRunnable;
   }
 
@@ -517,7 +513,7 @@ public class HiveSessionService extends BaseLensService implements SessionServic
      * Run internal.
      */
     public void runInternal() {
-      List<LensSessionHandle> sessionsToRemove = new ArrayList<LensSessionHandle>(SESSION_MAP.values());
+      List<LensSessionHandle> sessionsToRemove = new ArrayList<>(SESSION_MAP.values());
       Iterator<LensSessionHandle> itr = sessionsToRemove.iterator();
       while (itr.hasNext()) {
         LensSessionHandle sessionHandle = itr.next();
@@ -527,10 +523,12 @@ public class HiveSessionService extends BaseLensService implements SessionServic
             itr.remove();
           }
         } catch (ClientErrorException nfe) {
+          log.error("Error getting session " + sessionHandle.getPublicId(), nfe);
           itr.remove();
         }
       }
 
+      log.info("Sessions to remove : {} out of {} all sessions", sessionsToRemove.size(), SESSION_MAP.size());
       // Now close all inactive sessions
       for (LensSessionHandle sessionHandle : sessionsToRemove) {
         try {
@@ -540,6 +538,7 @@ public class HiveSessionService extends BaseLensService implements SessionServic
             + new Date(lastAccessTime));
           notifyEvent(new SessionExpired(System.currentTimeMillis(), sessionHandle));
         } catch (ClientErrorException nfe) {
+          log.error("Error getting session " + sessionHandle.getPublicId(), nfe);
           // Do nothing
         } catch (LensException e) {
           log.error("Error closing session " + sessionHandle.getPublicId() + " reason " + e.getMessage(), e);
@@ -555,9 +554,10 @@ public class HiveSessionService extends BaseLensService implements SessionServic
     @Override
     public void run() {
       try {
+        log.info("Running session expiry run");
         runInternal();
       } catch (Exception e) {
-        log.warn("Unknown error while checking for inactive sessions - " + e.getMessage());
+        log.warn("Unknown error while checking for inactive sessions - ", e);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lens/blob/98990c39/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 34c901c..08a5cff 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
@@ -62,9 +62,6 @@ public class LensSessionImpl extends HiveSessionImpl implements AutoCloseable {
   /** The persist info. */
   private LensSessionPersistInfo persistInfo = new LensSessionPersistInfo();
 
-  /** The last access time. */
-  private long lastAccessTime = System.currentTimeMillis();
-
   /** The session timeout. */
   private long sessionTimeout;
   private static class IntegerThreadLocal extends ThreadLocal<Integer> {
@@ -116,7 +113,7 @@ public class LensSessionImpl extends HiveSessionImpl implements AutoCloseable {
       getSessionHandle().getHandleIdentifier().getSecretId()));
     persistInfo.setUsername(getUserName());
     persistInfo.setPassword(getPassword());
-    persistInfo.setLastAccessTime(lastAccessTime);
+    persistInfo.setLastAccessTime(System.currentTimeMillis());
     persistInfo.setSessionConf(sessionConf);
     if (sessionConf != null) {
       for (Map.Entry<String, String> entry : sessionConf.entrySet()) {
@@ -280,12 +277,17 @@ public class LensSessionImpl extends HiveSessionImpl implements AutoCloseable {
   }
 
   public boolean isActive() {
-    return System.currentTimeMillis() - lastAccessTime < sessionTimeout
-      && (!persistInfo.markedForClose|| activeOperationsPresent());
+    // session is active, if any active operations are present.
+    // If no active operations are present, session is active if timeout is not reached and session is not
+    // marked for close
+    return activeOperationsPresent() || ((System.currentTimeMillis() - persistInfo.lastAccessTime < sessionTimeout)
+      && !persistInfo.markedForClose);
   }
+
   public boolean isMarkedForClose() {
     return persistInfo.isMarkedForClose();
   }
+
   public synchronized void setActive() {
     setLastAccessTime(System.currentTimeMillis());
   }
@@ -468,12 +470,12 @@ public class LensSessionImpl extends HiveSessionImpl implements AutoCloseable {
     return persistInfo;
   }
 
-  void setLastAccessTime(long lastAccessTime) {
-    this.lastAccessTime = lastAccessTime;
+  public void setLastAccessTime(long lastAccessTime) {
+    persistInfo.lastAccessTime = lastAccessTime;
   }
 
   public long getLastAccessTime() {
-    return lastAccessTime;
+    return persistInfo.lastAccessTime;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lens/blob/98990c39/lens-server/src/test/java/org/apache/lens/server/query/TestQueryIndependenceFromSessionClose.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryIndependenceFromSessionClose.java b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryIndependenceFromSessionClose.java
index 8c1bb7b..017584c 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryIndependenceFromSessionClose.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryIndependenceFromSessionClose.java
@@ -22,10 +22,7 @@ import static org.apache.lens.server.api.LensConfConstants.*;
 
 import static org.testng.Assert.*;
 
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 
 import javax.ws.rs.core.Application;
 import javax.ws.rs.core.MediaType;
@@ -33,6 +30,7 @@ import javax.ws.rs.core.Response;
 
 import org.apache.lens.api.LensConf;
 import org.apache.lens.api.LensSessionHandle;
+import org.apache.lens.api.query.LensQuery;
 import org.apache.lens.api.query.QueryHandle;
 import org.apache.lens.api.query.QueryStatus;
 import org.apache.lens.api.result.LensAPIResult;
@@ -41,6 +39,7 @@ import org.apache.lens.driver.hive.HiveDriver;
 import org.apache.lens.server.LensJerseyTest;
 import org.apache.lens.server.LensServerTestUtil;
 import org.apache.lens.server.LensServices;
+import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.LensServerAPITestUtil;
 import org.apache.lens.server.api.driver.LensDriver;
 import org.apache.lens.server.api.error.LensException;
@@ -51,6 +50,9 @@ import org.apache.lens.server.common.RestAPITestUtil;
 import org.apache.lens.server.common.TestResourceFile;
 import org.apache.lens.server.error.LensServerErrorCode;
 import org.apache.lens.server.session.HiveSessionService;
+import org.apache.lens.server.session.LensSessionImpl;
+
+import org.apache.hadoop.hive.conf.HiveConf;
 
 import org.glassfish.jersey.test.TestProperties;
 import org.testng.annotations.*;
@@ -111,10 +113,6 @@ public class TestQueryIndependenceFromSessionClose extends LensJerseyTest {
       QUERY_PERSISTENT_RESULT_INDRIVER, true,
       QUERY_OUTPUT_FORMATTER, TestQueryService.DeferredPersistentResultFormatter.class.getName());
   }
-  @AfterClass
-  public void restart() {
-    restartLensServer();
-  }
 
   @Override
   public Map<String, String> getServerConfOverWrites() {
@@ -150,10 +148,17 @@ public class TestQueryIndependenceFromSessionClose extends LensJerseyTest {
 
   private void customRestartLensServer() {
     queryService = null;
-    super.restartLensServer(getServerConf(), false);
+    super.restartLensServer(getServerConf());
     getQueryService();
   }
 
+  private void restartLensServerWithLowerExpiry() {
+    sessionService = null;
+    HiveConf hconf = new HiveConf(getServerConf());
+    hconf.setLong(LensConfConstants.SESSION_TIMEOUT_SECONDS, 1L);
+    super.restartLensServer(hconf);
+    getSessionService();
+  }
   /*
      * (non-Javadoc)
      *
@@ -277,6 +282,54 @@ public class TestQueryIndependenceFromSessionClose extends LensJerseyTest {
     return sessions;
   }
 
+  @Test
+  public void testSessionExpiryWithActiveOperation() throws Exception {
+    LensSessionHandle oldSession = getSession();
+    assertTrue(sessionService.getSession(oldSession).isActive());
+    restartLensServerWithLowerExpiry();
+    assertFalse(sessionService.getSession(oldSession).isActive());
+    // create a new session and launch a query
+    LensSessionHandle sessionHandle = getSession();
+    LensSessionImpl session = sessionService.getSession(sessionHandle);
+    QueryHandle handle = RestAPITestUtil.executeAndGetHandle(target(),
+      Optional.of(sessionHandle), Optional.of("select * from " + TEST_TABLE), Optional.of(conf), defaultMT);
+    assertTrue(session.isActive());
+    session.setLastAccessTime(
+      session.getLastAccessTime() - 2000 * getServerConf().getLong(LensConfConstants.SESSION_TIMEOUT_SECONDS,
+        LensConfConstants.SESSION_TIMEOUT_SECONDS_DEFAULT));
+    assertTrue(session.isActive());
+    assertFalse(session.isMarkedForClose());
+
+    LensSessionHandle sessionHandle2 = getSession();
+    LensQuery ctx = RestAPITestUtil.getLensQuery(target(), sessionHandle2, handle, defaultMT);
+    while (!ctx.getStatus().finished()) {
+      ctx = RestAPITestUtil.getLensQuery(target(), sessionHandle2, handle, defaultMT);
+      Thread.sleep(1000);
+      sessionHandle2 = getSession();
+    }
+    assertEquals(ctx.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL, String.valueOf(ctx));
+    assertFalse(session.isActive());
+    assertFalse(session.isMarkedForClose());
+
+    // run the expiry thread
+    sessionService.getSessionExpiryRunnable().run();
+    try {
+      sessionService.getSession(sessionHandle);
+      // should throw exception since session should be expired by now
+      fail("Expected get session to fail for session " + sessionHandle.getPublicId());
+    } catch (Exception e) {
+      // pass
+    }
+    try {
+      sessionService.getSession(oldSession);
+      // should throw exception since session should be expired by now
+      fail("Expected get session to fail for session " + oldSession.getPublicId());
+    } catch (Exception e) {
+      // pass
+    }
+    restartLensServer();
+    lensSessionId = getSession();
+  }
   @AfterMethod
   private void waitForPurge() throws InterruptedException {
     waitForPurge(0, getQueryService().finishedQueries);


[07/16] lens git commit: LENS-1380 : Revamp testcase division

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITCostTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITCostTests.java b/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITCostTests.java
index aeae8aa..1a7a391 100644
--- a/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITCostTests.java
+++ b/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITCostTests.java
@@ -19,7 +19,6 @@
 
 package org.apache.lens.regression.throttling;
 
-import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
@@ -37,7 +36,6 @@ import org.apache.lens.regression.core.helpers.*;
 import org.apache.lens.regression.core.testHelper.BaseTestClass;
 import org.apache.lens.regression.util.Util;
 import org.apache.lens.server.api.LensConfConstants;
-import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.util.LensUtil;
 
 import org.apache.log4j.Logger;
@@ -45,9 +43,6 @@ import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.*;
 
-import com.jcraft.jsch.JSchException;
-import com.jcraft.jsch.SftpException;
-
 public class ITCostTests extends BaseTestClass {
 
   private WebTarget servLens;
@@ -64,7 +59,7 @@ public class ITCostTests extends BaseTestClass {
   public static final String JDBC_QUERY1 = QueryInventory.getQueryFromInventory("JDBC.QUERY1");
 
   private static String hiveDriver = "hive/hive1";
-  private String hiveDriverSitePath  = lens.getServerDir() + "/conf/drivers/hive/hive1/hivedriver-site.xml";
+  private String lensSitePath  = lens.getServerDir() + "/conf/lens-site.xml";
   private static final long SECONDS_IN_A_MINUTE = 60;
 
   private static Logger logger = Logger.getLogger(ITCostTests.class);
@@ -75,6 +70,11 @@ public class ITCostTests extends BaseTestClass {
     logger.info("Creating a new Session");
     sessionHandleString = sHelper.openSession(lens.getCurrentDB());
     sHelper.setAndValidateParam(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "false");
+
+    HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.TOTAL_QUERY_COST_CEILING_PER_USER_KEY, "60",
+        LensConfConstants.QUERY_LAUNCHING_CONSTRAINT_FACTORIES_KEY, DriverConfig.USER_COST_CONSTRAINT_FACTORY);
+    Util.changeConfig(map, lensSitePath);
+    lens.restart();
   }
 
   @BeforeMethod(alwaysRun = true)
@@ -94,29 +94,10 @@ public class ITCostTests extends BaseTestClass {
   public void closeSession() throws Exception {
     logger.info("Closing Session");
     sHelper.closeSession();
-  }
-
-  @BeforeGroups("user-cost-ceiling")
-  public void setUserCeilingconfig() throws Exception {
-    try{
-      HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.TOTAL_QUERY_COST_CEILING_PER_USER_KEY, "60",
-          DriverConfig.HIVE_CONSTRAINT_FACTORIES,
-          DriverConfig.MAX_CONCURRENT_CONSTRAINT_FACTORY + "," + DriverConfig.USER_COST_CONSTRAINT_FACTORY,
-          DriverConfig.MAX_CONCURRENT_QUERIES, "10");
-      Util.changeConfig(map, hiveDriverSitePath);
-      lens.restart();
-    }catch (Exception e){
-      logger.info(e);
-    }
-  }
-
-  @AfterGroups("user-cost-ceiling")
-  public void restoreConfig() throws SftpException, JSchException, InterruptedException, LensException, IOException {
-    Util.changeConfig(hiveDriverSitePath);
+    Util.changeConfig(lensSitePath);
     lens.restart();
   }
 
-
   @Test(enabled = true, groups= "user-cost-ceiling")
   public void testUserCostCeiling() throws Exception {
 
@@ -293,148 +274,5 @@ public class ITCostTests extends BaseTestClass {
 
   //TODO : Add queue level throttling along with user ceiling constraint
 
-  /*
-  * LENS-995 : Queue number shouldn't change with in the same prority
-  */
-
-  @Test(enabled = true)
-  public void queueNumberChangeWithInSamePriority() throws Exception {
-
-    String longRunningQuery = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_95"), "20");
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "1");
-    String[] queries = {longRunningQuery, COST_5, COST_5, COST_3, COST_2};
-
-    try {
-      Util.changeConfig(map, hiveDriverSitePath);
-      lens.restart();
-
-      List<QueryHandle> handleList = new ArrayList<>();
-      for(String query : queries){
-        handleList.add((QueryHandle) qHelper.executeQuery(query).getData());
-      }
-
-      LensQuery lq1 = qHelper.getLensQuery(sessionHandleString, handleList.get(1));
-      LensQuery lq2 = qHelper.getLensQuery(sessionHandleString, handleList.get(2));
-      LensQuery lq3 = qHelper.getLensQuery(sessionHandleString, handleList.get(3));
-      LensQuery lq4 = qHelper.getLensQuery(sessionHandleString, handleList.get(4));
-
-      Assert.assertEquals(lq1.getStatus().getQueueNumber().intValue(), 1);
-      Assert.assertEquals(lq2.getStatus().getQueueNumber().intValue(), 2);
-      Assert.assertEquals(lq3.getStatus().getQueueNumber().intValue(), 3);
-      Assert.assertEquals(lq4.getStatus().getQueueNumber().intValue(), 4);
-
-      LensQuery lq0 = qHelper.waitForCompletion(handleList.get(0));
-
-      lq1 = qHelper.getLensQuery(sessionHandleString, handleList.get(1));
-      lq2 = qHelper.getLensQuery(sessionHandleString, handleList.get(2));
-      lq3 = qHelper.getLensQuery(sessionHandleString, handleList.get(3));
-      lq4 = qHelper.getLensQuery(sessionHandleString, handleList.get(4));
-
-      Assert.assertEquals(lq0.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-      Assert.assertEquals(lq2.getStatus().getQueueNumber().intValue(), 1);
-      Assert.assertEquals(lq3.getStatus().getQueueNumber().intValue(), 2);
-      Assert.assertEquals(lq4.getStatus().getQueueNumber().intValue(), 3);
-
-      lq1 = qHelper.waitForCompletion(handleList.get(1));
-
-      lq2 = qHelper.getLensQuery(sessionHandleString, handleList.get(2));
-      lq3 = qHelper.getLensQuery(sessionHandleString, handleList.get(3));
-      lq4 = qHelper.getLensQuery(sessionHandleString, handleList.get(4));
-
-      Assert.assertEquals(lq1.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-      Assert.assertEquals(lq3.getStatus().getQueueNumber().intValue(), 1);
-      Assert.assertEquals(lq4.getStatus().getQueueNumber().intValue(), 2);
-
-    }finally {
-      Util.changeConfig(hiveDriverSitePath);
-      lens.restart();
-    }
-  }
-
-
-  @Test(enabled = true)
-  public void queueNumberChangeDifferentPriority() throws Exception {
-
-    String longRunningQuery = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_95"), "20");
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "1");
-    try {
-      Util.changeConfig(map, hiveDriverSitePath);
-      lens.restart();
-
-      QueryHandle q0 = (QueryHandle) qHelper.executeQuery(longRunningQuery).getData();
-      QueryHandle q1 = (QueryHandle) qHelper.executeQuery(COST_20).getData();
-      QueryHandle q2 = (QueryHandle) qHelper.executeQuery(COST_2).getData();
-
-      LensQuery normal1 = qHelper.getLensQuery(sessionHandleString, q1);
-      LensQuery high1 = qHelper.getLensQuery(sessionHandleString, q2);
-
-      Assert.assertEquals(normal1.getStatus().getQueueNumber().intValue(), 2);
-      Assert.assertEquals(high1.getStatus().getQueueNumber().intValue(), 1);
-
-      QueryHandle q3 = (QueryHandle) qHelper.executeQuery(COST_5).getData();
-
-      LensQuery high2 = qHelper.getLensQuery(sessionHandleString, q3);
-      high1 = qHelper.getLensQuery(sessionHandleString, q2);
-      normal1 = qHelper.getLensQuery(sessionHandleString, q1);
-
-      Assert.assertEquals(normal1.getStatus().getQueueNumber().intValue(), 3);
-      Assert.assertEquals(high1.getStatus().getQueueNumber().intValue(), 1);
-      Assert.assertEquals(high2.getStatus().getQueueNumber().intValue(), 2);
-
-      QueryHandle q4 = (QueryHandle) qHelper.executeQuery(COST_20).getData();
-
-      LensQuery normal2 = qHelper.getLensQuery(sessionHandleString, q4);
-      normal1 = qHelper.getLensQuery(sessionHandleString, q1);
-      high1 = qHelper.getLensQuery(sessionHandleString, q2);
-      high2 = qHelper.getLensQuery(sessionHandleString, q3);
-
-      Assert.assertEquals(high1.getStatus().getQueueNumber().intValue(), 1);
-      Assert.assertEquals(high2.getStatus().getQueueNumber().intValue(), 2);
-      Assert.assertEquals(normal1.getStatus().getQueueNumber().intValue(), 3);
-      Assert.assertEquals(normal2.getStatus().getQueueNumber().intValue(), 4);
-
-    }finally {
-      Util.changeConfig(hiveDriverSitePath);
-      lens.restart();
-    }
-  }
-
-
-  @Test(enabled = true)
-  public void queueNumberChangeDifferentPriorityWithJdbc() throws Exception {
-
-    String longRunningQuery = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_95"), "20");
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "1");
-    List<QueryHandle> handleList = new ArrayList<>();
-
-    try {
-      Util.changeConfig(map, hiveDriverSitePath);
-      lens.restart();
-
-      String[] queries = {COST_20, COST_2, COST_3, COST_60, COST_5, COST_10, COST_3};
-      // Queue order is determined from priority and order in which queries are fired.
-      int[] queueNo = {5, 1, 2, 7, 3, 6, 4};
-
-      qHelper.executeQuery(longRunningQuery);
-      for(String query : queries){
-        handleList.add((QueryHandle) qHelper.executeQuery(query).getData());
-        qHelper.executeQuery(JDBC_QUERY1).getData();
-      }
-
-      List<LensQuery> lqList = new ArrayList<>();
-      for(QueryHandle qh : handleList){
-        lqList.add(qHelper.getLensQuery(sessionHandleString, qh));
-      }
-
-      for(int i = 0; i < lqList.size(); i++) {
-        Assert.assertEquals(lqList.get(i).getStatus().getQueueNumber().intValue(), queueNo[i]);
-      }
-
-    }finally {
-      Util.changeConfig(hiveDriverSitePath);
-      lens.restart();
-    }
-  }
-
 }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITQueueNumberTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITQueueNumberTests.java b/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITQueueNumberTests.java
new file mode 100644
index 0000000..b7cea6b
--- /dev/null
+++ b/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITQueueNumberTests.java
@@ -0,0 +1,232 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.lens.regression.throttling;
+
+import java.lang.reflect.Method;
+import java.util.*;
+
+import javax.ws.rs.client.WebTarget;
+
+import org.apache.lens.api.query.LensQuery;
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+import org.apache.lens.cube.parse.CubeQueryConfUtil;
+import org.apache.lens.regression.core.constants.DriverConfig;
+import org.apache.lens.regression.core.constants.QueryInventory;
+import org.apache.lens.regression.core.helpers.ServiceManagerHelper;
+import org.apache.lens.regression.core.testHelper.BaseTestClass;
+import org.apache.lens.regression.util.Util;
+import org.apache.lens.server.api.util.LensUtil;
+
+import org.apache.log4j.Logger;
+
+import org.testng.Assert;
+import org.testng.annotations.*;
+
+public class ITQueueNumberTests extends BaseTestClass {
+
+  private WebTarget servLens;
+  private String sessionHandleString;
+
+  public static final String COST_95 = QueryInventory.getQueryFromInventory("HIVE.COST_95");
+  public static final String COST_60 = QueryInventory.getQueryFromInventory("HIVE.COST_60");
+  public static final String COST_30 = QueryInventory.getQueryFromInventory("HIVE.COST_30");
+  public static final String COST_20 = QueryInventory.getQueryFromInventory("HIVE.COST_20");
+  public static final String COST_10 = QueryInventory.getQueryFromInventory("HIVE.COST_10");
+  public static final String COST_5 = QueryInventory.getQueryFromInventory("HIVE.COST_5");
+  public static final String COST_3 = QueryInventory.getQueryFromInventory("HIVE.COST_3");
+  public static final String COST_2 = QueryInventory.getQueryFromInventory("HIVE.COST_2");
+  public static final String JDBC_QUERY1 = QueryInventory.getQueryFromInventory("JDBC.QUERY1");
+
+  private String hiveDriverSitePath  = lens.getServerDir() + "/conf/drivers/hive/hive1/hivedriver-site.xml";
+  private static Logger logger = Logger.getLogger(ITQueueNumberTests.class);
+
+  @BeforeClass(alwaysRun = true)
+  public void initialize() throws Exception {
+    servLens = ServiceManagerHelper.init();
+    logger.info("Creating a new Session");
+    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+    sHelper.setAndValidateParam(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "false");
+  }
+
+  @BeforeMethod(alwaysRun = true)
+  public void setUp(Method method) throws Exception {
+    logger.info("Test Name: " + method.getName());
+  }
+
+  @AfterMethod(alwaysRun = true)
+  public void afterMethod(Method method) throws Exception {
+    logger.info("Test Name: " + method.getName());
+    qHelper.killQuery(null, "QUEUED", "all");
+    qHelper.killQuery(null, "RUNNING", "all");
+    qHelper.killQuery(null, "EXECUTED", "all");
+  }
+
+  @AfterClass(alwaysRun = true)
+  public void closeSession() throws Exception {
+    logger.info("Closing Session");
+    sHelper.closeSession();
+  }
+
+
+  /*
+  * LENS-995 : Queue number shouldn't change with in the same prority
+  */
+
+  @Test(enabled = true)
+  public void queueNumberChangeWithInSamePriority() throws Exception {
+
+    String longRunningQuery = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_95"), "20");
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "1");
+    String[] queries = {longRunningQuery, COST_5, COST_5, COST_3, COST_2};
+
+    try {
+      Util.changeConfig(map, hiveDriverSitePath);
+      lens.restart();
+
+      List<QueryHandle> handleList = new ArrayList<>();
+      for(String query : queries){
+        handleList.add((QueryHandle) qHelper.executeQuery(query).getData());
+      }
+
+      LensQuery lq1 = qHelper.getLensQuery(sessionHandleString, handleList.get(1));
+      LensQuery lq2 = qHelper.getLensQuery(sessionHandleString, handleList.get(2));
+      LensQuery lq3 = qHelper.getLensQuery(sessionHandleString, handleList.get(3));
+      LensQuery lq4 = qHelper.getLensQuery(sessionHandleString, handleList.get(4));
+
+      Assert.assertEquals(lq1.getStatus().getQueueNumber().intValue(), 1);
+      Assert.assertEquals(lq2.getStatus().getQueueNumber().intValue(), 2);
+      Assert.assertEquals(lq3.getStatus().getQueueNumber().intValue(), 3);
+      Assert.assertEquals(lq4.getStatus().getQueueNumber().intValue(), 4);
+
+      LensQuery lq0 = qHelper.waitForCompletion(handleList.get(0));
+
+      lq1 = qHelper.getLensQuery(sessionHandleString, handleList.get(1));
+      lq2 = qHelper.getLensQuery(sessionHandleString, handleList.get(2));
+      lq3 = qHelper.getLensQuery(sessionHandleString, handleList.get(3));
+      lq4 = qHelper.getLensQuery(sessionHandleString, handleList.get(4));
+
+      Assert.assertEquals(lq0.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+      Assert.assertEquals(lq2.getStatus().getQueueNumber().intValue(), 1);
+      Assert.assertEquals(lq3.getStatus().getQueueNumber().intValue(), 2);
+      Assert.assertEquals(lq4.getStatus().getQueueNumber().intValue(), 3);
+
+      lq1 = qHelper.waitForCompletion(handleList.get(1));
+
+      lq2 = qHelper.getLensQuery(sessionHandleString, handleList.get(2));
+      lq3 = qHelper.getLensQuery(sessionHandleString, handleList.get(3));
+      lq4 = qHelper.getLensQuery(sessionHandleString, handleList.get(4));
+
+      Assert.assertEquals(lq1.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+      Assert.assertEquals(lq3.getStatus().getQueueNumber().intValue(), 1);
+      Assert.assertEquals(lq4.getStatus().getQueueNumber().intValue(), 2);
+
+    }finally {
+      Util.changeConfig(hiveDriverSitePath);
+      lens.restart();
+    }
+  }
+
+
+  @Test(enabled = true)
+  public void queueNumberChangeDifferentPriority() throws Exception {
+
+    String longRunningQuery = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_95"), "20");
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "1");
+    try {
+      Util.changeConfig(map, hiveDriverSitePath);
+      lens.restart();
+
+      QueryHandle q0 = (QueryHandle) qHelper.executeQuery(longRunningQuery).getData();
+      QueryHandle q1 = (QueryHandle) qHelper.executeQuery(COST_20).getData();
+      QueryHandle q2 = (QueryHandle) qHelper.executeQuery(COST_2).getData();
+
+      LensQuery normal1 = qHelper.getLensQuery(sessionHandleString, q1);
+      LensQuery high1 = qHelper.getLensQuery(sessionHandleString, q2);
+
+      Assert.assertEquals(normal1.getStatus().getQueueNumber().intValue(), 2);
+      Assert.assertEquals(high1.getStatus().getQueueNumber().intValue(), 1);
+
+      QueryHandle q3 = (QueryHandle) qHelper.executeQuery(COST_5).getData();
+
+      LensQuery high2 = qHelper.getLensQuery(sessionHandleString, q3);
+      high1 = qHelper.getLensQuery(sessionHandleString, q2);
+      normal1 = qHelper.getLensQuery(sessionHandleString, q1);
+
+      Assert.assertEquals(normal1.getStatus().getQueueNumber().intValue(), 3);
+      Assert.assertEquals(high1.getStatus().getQueueNumber().intValue(), 1);
+      Assert.assertEquals(high2.getStatus().getQueueNumber().intValue(), 2);
+
+      QueryHandle q4 = (QueryHandle) qHelper.executeQuery(COST_20).getData();
+
+      LensQuery normal2 = qHelper.getLensQuery(sessionHandleString, q4);
+      normal1 = qHelper.getLensQuery(sessionHandleString, q1);
+      high1 = qHelper.getLensQuery(sessionHandleString, q2);
+      high2 = qHelper.getLensQuery(sessionHandleString, q3);
+
+      Assert.assertEquals(high1.getStatus().getQueueNumber().intValue(), 1);
+      Assert.assertEquals(high2.getStatus().getQueueNumber().intValue(), 2);
+      Assert.assertEquals(normal1.getStatus().getQueueNumber().intValue(), 3);
+      Assert.assertEquals(normal2.getStatus().getQueueNumber().intValue(), 4);
+
+    }finally {
+      Util.changeConfig(hiveDriverSitePath);
+      lens.restart();
+    }
+  }
+
+
+  @Test(enabled = true)
+  public void queueNumberChangeDifferentPriorityWithJdbc() throws Exception {
+
+    String longRunningQuery = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_95"), "20");
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "1");
+    List<QueryHandle> handleList = new ArrayList<>();
+
+    try {
+      Util.changeConfig(map, hiveDriverSitePath);
+      lens.restart();
+
+      String[] queries = {COST_20, COST_2, COST_3, COST_60, COST_5, COST_10, COST_3};
+      // Queue order is determined from priority and order in which queries are fired.
+      int[] queueNo = {5, 1, 2, 7, 3, 6, 4};
+
+      qHelper.executeQuery(longRunningQuery);
+      for(String query : queries){
+        handleList.add((QueryHandle) qHelper.executeQuery(query).getData());
+        qHelper.executeQuery(JDBC_QUERY1).getData();
+      }
+
+      List<LensQuery> lqList = new ArrayList<>();
+      for(QueryHandle qh : handleList){
+        lqList.add(qHelper.getLensQuery(sessionHandleString, qh));
+      }
+
+      for(int i = 0; i < lqList.size(); i++) {
+        Assert.assertEquals(lqList.get(i).getStatus().getQueueNumber().intValue(), queueNo[i]);
+      }
+
+    }finally {
+      Util.changeConfig(hiveDriverSitePath);
+      lens.restart();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITThrottlingTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITThrottlingTests.java b/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITThrottlingTests.java
new file mode 100644
index 0000000..6e34cb5
--- /dev/null
+++ b/lens-regression/src/test/java/org/apache/lens/regression/throttling/ITThrottlingTests.java
@@ -0,0 +1,605 @@
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.lens.regression.throttling;
+
+import java.lang.reflect.Method;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+
+import javax.ws.rs.client.WebTarget;
+
+import org.apache.lens.api.Priority;
+import org.apache.lens.api.query.LensQuery;
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+import org.apache.lens.cube.parse.CubeQueryConfUtil;
+import org.apache.lens.driver.hive.HiveDriver;
+import org.apache.lens.regression.core.constants.DriverConfig;
+import org.apache.lens.regression.core.constants.QueryInventory;
+import org.apache.lens.regression.core.helpers.ServiceManagerHelper;
+import org.apache.lens.regression.core.testHelper.BaseTestClass;
+import org.apache.lens.regression.util.Util;
+import org.apache.lens.server.api.LensConfConstants;
+import org.apache.lens.server.api.util.LensUtil;
+
+import org.apache.log4j.Logger;
+
+import org.testng.Assert;
+import org.testng.annotations.*;
+
+public class ITThrottlingTests extends BaseTestClass {
+
+  WebTarget servLens;
+  String sessionHandleString;
+
+  public static final String SLEEP_QUERY = QueryInventory.getSleepQuery("5");
+  public static final String COST_95 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_95"), "5");
+  public static final String COST_60 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_60"), "5");
+  public static final String COST_30 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_30"), "5");
+  public static final String COST_20 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_20"), "4");
+  public static final String COST_10 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_10"), "4");
+  public static final String COST_5 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_5"), "3");
+  public static final String JDBC_QUERY1 = QueryInventory.getQueryFromInventory("JDBC.QUERY1");
+
+  private static String hiveDriver = "hive/hive1";
+  private final String hiveDriverConf = lens.getServerDir() + "/conf/drivers/hive/hive1/hivedriver-site.xml";
+  private final String backupConfFilePath = lens.getServerDir() + "/conf/drivers/hive/hive1/backup-hivedriver-site.xml";
+
+  private static final long SECONDS_IN_A_MINUTE = 60;
+  private String session1 = null, session2 = null;
+  //TODO : Read queue names from property file
+  private static String queue1 = "dwh", queue2 = "reports";
+
+  private static Logger logger = Logger.getLogger(ITThrottlingTests.class);
+
+  @BeforeClass(alwaysRun = true)
+  public void initialize() throws Exception {
+    servLens = ServiceManagerHelper.init();
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "10",
+        HiveDriver.HS2_PRIORITY_RANGES, "HIGH,7,NORMAL,30,LOW,90,VERY_LOW");
+    Util.changeConfig(map, hiveDriverConf);
+    lens.restart();
+  }
+
+  @BeforeMethod(alwaysRun = true)
+  public void setUp(Method method) throws Exception {
+    logger.info("Test Name: " + method.getName());
+    Util.runRemoteCommand("cp " + hiveDriverConf + " " + backupConfFilePath);
+
+    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+    session1 = sHelper.openSession("diff1", "diff1", lens.getCurrentDB());
+    session2 = sHelper.openSession("diff2", "diff2", lens.getCurrentDB());
+    sHelper.setAndValidateParam(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "false");
+    sHelper.setAndValidateParam(session1, CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "false");
+    sHelper.setAndValidateParam(session2, CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "false");
+  }
+
+  @AfterMethod(alwaysRun = true)
+  public void afterMethod(Method method) throws Exception {
+    logger.info("Test Name: " + method.getName());
+    qHelper.killQuery(null, "QUEUED", "all");
+    qHelper.killQuery(null, "RUNNING", "all");
+    qHelper.killQuery(null, "EXECUTED", "all");
+    sHelper.closeSession(session1);
+    sHelper.closeSession(session2);
+    sHelper.closeSession(sessionHandleString);
+
+    Util.runRemoteCommand("cp " + backupConfFilePath + " " + hiveDriverConf);
+  }
+
+  @AfterClass(alwaysRun = true)
+  public void closeSession() throws Exception {
+    lens.restart();
+  }
+
+
+  @Test(enabled = true)
+  public void testHiveThrottling() throws Exception {
+
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "2");
+    Util.changeConfig(map, hiveDriverConf);
+    lens.restart();
+
+    List<QueryHandle> handleList = new ArrayList<>();
+    handleList.add((QueryHandle) qHelper.executeQuery(SLEEP_QUERY).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(SLEEP_QUERY, null, session1).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(SLEEP_QUERY, null, session2).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(SLEEP_QUERY, null).getData());
+
+    Thread.sleep(1000);
+
+    List<QueryStatus> statusList = new ArrayList<>();
+    for(QueryHandle handle : handleList){
+      statusList.add(qHelper.getQueryStatus(handle));
+    }
+
+    Assert.assertEquals(statusList.get(0).getStatus(), QueryStatus.Status.RUNNING);
+    Assert.assertEquals(statusList.get(1).getStatus(), QueryStatus.Status.RUNNING);
+    Assert.assertEquals(statusList.get(2).getStatus(), QueryStatus.Status.QUEUED);
+    Assert.assertEquals(statusList.get(3).getStatus(), QueryStatus.Status.QUEUED);
+
+    qHelper.waitForCompletion(handleList.get(0));
+    Thread.sleep(100);
+    Assert.assertEquals(qHelper.getQueryStatus(handleList.get(2)).getStatus(), QueryStatus.Status.RUNNING);
+  }
+
+
+  @Test(enabled = true)
+  public void testHiveMaxConcurrentRandomQueryIngestion() throws Exception {
+
+    long timeToWait= 5 * SECONDS_IN_A_MINUTE; // in seconds
+    int sleepTime = 3, maxConcurrent = 4;
+    List<QueryHandle> handleList = new ArrayList<QueryHandle>();
+
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
+        String.valueOf(maxConcurrent));
+    Util.changeConfig(map, hiveDriverConf);
+    lens.restart();
+
+    for (int i = 0; i < 5; i++) {
+      handleList.add((QueryHandle) qHelper.executeQuery(SLEEP_QUERY).getData());
+      handleList.add((QueryHandle) qHelper.executeQuery(JDBC_QUERY1).getData());
+      handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_CUBE_QUERY, null, session1).getData());
+    }
+
+    Thread.sleep(50);
+
+    List<QueryHandle> running = null, queued = null;
+    for (int t = 0; t < timeToWait; t = t + sleepTime) {
+
+      running = qHelper.getQueryHandleList(null, "RUNNING", "all", sessionHandleString, null, null, hiveDriver);
+      queued = qHelper.getQueryHandleList(null, "QUEUED", "all", sessionHandleString, null, null, hiveDriver);
+      logger.info("Running query count : " + running.size() + "\t Queued query count : " + queued.size());
+
+      if (running.isEmpty() && queued.isEmpty()) {
+        break;
+      }
+      Assert.assertTrue(running.size() <= maxConcurrent);
+
+      if (t % 30 == 0 && t < 200) {
+        handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_DIM_QUERY).getData());
+        handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.JDBC_CUBE_QUERY).getData());
+        handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_CUBE_QUERY, null, session1).getData());
+      }
+      TimeUnit.SECONDS.sleep(sleepTime);
+    }
+
+    Assert.assertTrue(running.isEmpty());
+    Assert.assertTrue(queued.isEmpty());
+
+    for(QueryHandle q : handleList){
+      LensQuery lq = qHelper.waitForCompletion(q);
+      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+    }
+  }
+
+  @Test(enabled = true)
+  public void testProrityMaxConcurrent() throws Exception {
+
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "5",
+        DriverConfig.PRIORITY_MAX_CONCURRENT, "HIGH=2,VERY_LOW=1", HiveDriver.HS2_PRIORITY_RANGES,
+        "HIGH,7,NORMAL,30,LOW,90,VERY_LOW");
+    Util.changeConfig(map, hiveDriverConf);
+    lens.restart();
+
+    /* First 3 are high priority queries, 2 of them will go to RUNNING, 3rd will be queued as there is a
+      threshold of 2 on HIGH priority. cost_95 queries are very_low priority ones, hence 1st will go to RUNNING,
+      other 1 is queued. cost_20 and cost_60 goes to running as they are normal and low priority queries and there
+      is no limit set for low and normal priority.Last cost_60 query goes to queue as, RUNNING query on this
+      driver has reached max concurrent threshold.
+    */
+
+    String[] queries = {COST_5, COST_5, COST_5, COST_95, COST_95, COST_20, COST_60, COST_60};
+    QueryStatus.Status[] expectedStatus = {QueryStatus.Status.RUNNING, QueryStatus.Status.RUNNING,
+      QueryStatus.Status.QUEUED, QueryStatus.Status.RUNNING, QueryStatus.Status.QUEUED,
+      QueryStatus.Status.RUNNING, QueryStatus.Status.RUNNING, QueryStatus.Status.QUEUED, };
+
+    List<QueryHandle> handleList = new ArrayList<>();
+    for (String query : queries){
+      handleList.add((QueryHandle) qHelper.executeQuery(query).getData());
+    }
+
+    List<QueryStatus.Status> statusList = new ArrayList<>();
+    for (QueryHandle handle : handleList){
+      statusList.add(qHelper.getQueryStatus(handle).getStatus());
+    }
+
+    for (int i=0; i<statusList.size(); i++){
+      Assert.assertEquals(statusList.get(i), expectedStatus[i]);
+    }
+
+    qHelper.waitForCompletion(handleList.get(0));
+    Assert.assertEquals(qHelper.getQueryStatus(handleList.get(2)).getStatus(), QueryStatus.Status.RUNNING);
+  }
+
+  @Test(enabled = true)
+  public void prioritySumMoreThanMaxConcurrent() throws Exception {
+
+    long timeToWait= 5 * SECONDS_IN_A_MINUTE; // in seconds
+    int sleepTime = 5, maxConcurrent = 5, lowConCurrent = 2, veryLowConcurrent = 1, highConcurrent = 4,
+        normalConcurrent = 2;
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
+        String.valueOf(maxConcurrent), DriverConfig.PRIORITY_MAX_CONCURRENT,
+        "LOW=" + String.valueOf(lowConCurrent) + ",VERY_LOW=" + String.valueOf(veryLowConcurrent)
+        + ",NORMAL=" + String.valueOf(normalConcurrent) + ",HIGH=" + String.valueOf(highConcurrent));
+    Util.changeConfig(map, hiveDriverConf);
+    lens.restart();
+
+    List<QueryHandle> handleList = new ArrayList<QueryHandle>();
+    for (int i=1; i<=5; i++){
+      handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
+      handleList.add((QueryHandle) qHelper.executeQuery(COST_60, null, session1).getData());
+      handleList.add((QueryHandle) qHelper.executeQuery(COST_20, null, session2).getData());
+      handleList.add((QueryHandle) qHelper.executeQuery(COST_5).getData());
+    }
+
+    List<QueryHandle> running=null, queued=null;
+    for (int t = 0; t < timeToWait; t = t + sleepTime) {
+
+      running = qHelper.getQueryHandleList(null, "RUNNING", "all", sessionHandleString, null, null, hiveDriver);
+      queued = qHelper.getQueryHandleList(null, "QUEUED", "all", sessionHandleString, null, null, hiveDriver);
+      logger.info("Running query count : " + running.size() + "\t Queued query count : " + queued.size());
+
+      if (running.isEmpty() && queued.isEmpty()) {
+        break;
+      }
+
+      Assert.assertTrue(running.size() <= maxConcurrent);
+
+      int low = 0, veryLow = 0, high = 0, normal = 0;
+      for (QueryHandle qh : running) {
+        Priority p = qHelper.getLensQuery(sessionHandleString, qh).getPriority();
+        Assert.assertNotNull(p);
+        switch (p) {
+        case HIGH:
+          high++;
+          break;
+        case NORMAL:
+          normal++;
+          break;
+        case LOW:
+          low++;
+          break;
+        case VERY_LOW:
+          veryLow++;
+          break;
+        default:
+          throw new Exception("Unexpected Priority");
+        }
+      }
+
+      Assert.assertTrue(low <= lowConCurrent);
+      Assert.assertTrue(veryLow <= veryLowConcurrent);
+      Assert.assertTrue(high <= highConcurrent);
+      Assert.assertTrue(normal <= normalConcurrent);
+
+      TimeUnit.SECONDS.sleep(sleepTime);
+    }
+
+    Assert.assertTrue(queued.isEmpty());
+    Assert.assertTrue(running.isEmpty());
+
+    for (QueryHandle q: handleList){
+      LensQuery lq = qHelper.waitForCompletion(q);
+      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+    }
+  }
+
+
+  @Test(enabled = true)
+  public void queueMaxConcurrent() throws Exception {
+
+    int maxConcurrent = 3, queue1Count = 1, queue2Count = 2;
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
+        String.valueOf(maxConcurrent), DriverConfig.QUEUE_MAX_CONCURRENT,
+        queue1 + "=" + String.valueOf(queue1Count) + "," + queue2 + "=" + String.valueOf(queue2Count));
+    List<QueryHandle> handleList = new ArrayList<>();
+
+    Util.changeConfig(map, hiveDriverConf);
+    lens.restart();
+
+    sHelper.setAndValidateParam(LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue1);
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_60).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
+
+    sHelper.setAndValidateParam(LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_60).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
+
+    List<QueryStatus.Status> statusList = new ArrayList<>();
+    for (QueryHandle handle : handleList){
+      statusList.add(qHelper.getQueryStatus(handle).getStatus());
+    }
+
+    Assert.assertEquals(statusList.get(0), QueryStatus.Status.RUNNING);
+    Assert.assertEquals(statusList.get(1), QueryStatus.Status.QUEUED);
+    Assert.assertEquals(statusList.get(2), QueryStatus.Status.RUNNING);
+    Assert.assertEquals(statusList.get(3), QueryStatus.Status.RUNNING);
+    Assert.assertEquals(statusList.get(4), QueryStatus.Status.QUEUED);
+
+    qHelper.waitForCompletion(handleList.get(0));
+    Assert.assertEquals(qHelper.getQueryStatus(handleList.get(1)).getStatus(), QueryStatus.Status.RUNNING);
+
+    qHelper.waitForCompletion(handleList.get(2));
+    Assert.assertEquals(qHelper.getQueryStatus(handleList.get(4)).getStatus(), QueryStatus.Status.RUNNING);
+  }
+
+  // LENS-1027
+  @Test(enabled = true)
+  public void queueDefaultThresholdConstraint() throws Exception {
+
+    int maxConcurrent = 5, queue1Count = 1, queue2Count = 2;
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
+        String.valueOf(maxConcurrent), DriverConfig.QUEUE_MAX_CONCURRENT,
+        "*=" + String.valueOf(queue1Count) + "," + queue2 + "=" + String.valueOf(queue2Count));
+    List<QueryHandle> handleList = new ArrayList<>();
+
+    Util.changeConfig(map, hiveDriverConf);
+    lens.restart();
+
+    sHelper.setAndValidateParam(LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue1);
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_60).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
+
+    sHelper.setAndValidateParam(LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_60).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
+
+    Thread.sleep(2000);
+
+    List<QueryStatus.Status> statusList = new ArrayList<>();
+    for (QueryHandle handle : handleList){
+      statusList.add(qHelper.getQueryStatus(handle).getStatus());
+    }
+
+    Assert.assertEquals(statusList.get(0), QueryStatus.Status.RUNNING);
+    Assert.assertEquals(statusList.get(1), QueryStatus.Status.QUEUED);
+    Assert.assertEquals(statusList.get(2), QueryStatus.Status.RUNNING);
+    Assert.assertEquals(statusList.get(3), QueryStatus.Status.RUNNING);
+    Assert.assertEquals(statusList.get(4), QueryStatus.Status.QUEUED);
+  }
+
+
+  @Test(enabled = true)
+  public void enableQueueThrottlingWithExistingQueuedQueries() throws Exception {
+
+    long timeToWait= 5 * SECONDS_IN_A_MINUTE; // in seconds
+    int sleepTime = 5, maxConcurrent = 4, queue1Concurrent = 1, queue2Concurrent = 2;
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
+        String.valueOf(maxConcurrent), DriverConfig.QUEUE_MAX_CONCURRENT,
+        queue1 + "=" + String.valueOf(queue1Concurrent) + "," + queue2 + "=" + String.valueOf(queue2Concurrent));
+
+    sHelper.setAndValidateParam(session1, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue1);
+    sHelper.setAndValidateParam(session2, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
+
+    List<QueryHandle> handleList = new ArrayList<QueryHandle>();
+    for (int i = 1; i <= 3; i++) {
+      handleList.add((QueryHandle)qHelper.executeQuery(COST_95).getData());
+      handleList.add((QueryHandle)qHelper.executeQuery(COST_20, "", session1).getData());
+      handleList.add((QueryHandle)qHelper.executeQuery(COST_95, "", session2).getData());
+    }
+
+    Util.changeConfig(map, hiveDriverConf);
+    lens.restart();
+
+    for (int i = 1; i <= 2; i++) {
+      handleList.add((QueryHandle)qHelper.executeQuery(COST_95).getData());
+      handleList.add((QueryHandle)qHelper.executeQuery(COST_20, "", session1).getData());
+      handleList.add((QueryHandle)qHelper.executeQuery(COST_95, "", session2).getData());
+    }
+
+    List<QueryHandle> running = null, queued = null;
+    for (int t = 0; t < timeToWait; t = t + sleepTime) {
+
+      running = qHelper.getQueryHandleList(null, "RUNNING", "all", sessionHandleString, null, null, hiveDriver);
+      queued = qHelper.getQueryHandleList(null, "QUEUED", "all", sessionHandleString,  null, null, hiveDriver);
+      logger.info("Running query count : " + running.size() + "\t Queued query count : " + queued.size());
+
+      if (running.isEmpty() && queued.isEmpty()) {
+        break;
+      }
+      Assert.assertTrue(running.size() <= maxConcurrent, "running query count is not less than max concurrent set"
+          + "running-count : " + running.size() + ", max-count : " + maxConcurrent);
+
+      int queue1Count = 0, queue2Count = 0;
+      for (QueryHandle qh : running) {
+        String queue = qHelper.getLensQuery(sessionHandleString, qh).getQueryConf().getProperties()
+            .get("mapreduce.job.queuename");
+        Assert.assertNotNull(queue);
+
+        if (queue.equals(queue1)) {
+          queue1Count++;
+        } else if (queue.equals(queue2)) {
+          queue2Count++;
+        }
+      }
+
+      Assert.assertTrue(queue1Count <= queue1Concurrent, "queue1 count : " + queue1Count);
+      Assert.assertTrue(queue2Count <= queue2Concurrent, "queue2 count : " + queue2Count);
+      TimeUnit.SECONDS.sleep(sleepTime);
+    }
+
+    Assert.assertTrue(running.isEmpty());
+    Assert.assertTrue(queued.isEmpty());
+
+    for(QueryHandle q: handleList){
+      LensQuery lq = qHelper.waitForCompletion(q);
+      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+    }
+  }
+
+
+  @Test(enabled = true)
+  public void queueAndPriorityMaxConcurrent() throws Exception {
+
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "5",
+      DriverConfig.PRIORITY_MAX_CONCURRENT, "LOW=2,VERY_LOW=1",
+      DriverConfig.QUEUE_MAX_CONCURRENT, queue1 + "=1," + queue2 + "=2");
+
+    Util.changeConfig(map, hiveDriverConf);
+    lens.restart();
+
+    sHelper.setAndValidateParam(session1, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue1);
+    sHelper.setAndValidateParam(session2, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
+
+    QueryStatus.Status[] expectedStatus = {QueryStatus.Status.RUNNING, QueryStatus.Status.QUEUED,
+      QueryStatus.Status.QUEUED, QueryStatus.Status.RUNNING, QueryStatus.Status.RUNNING,
+      QueryStatus.Status.RUNNING, QueryStatus.Status.QUEUED, QueryStatus.Status.QUEUED, };
+
+    List<QueryHandle> handleList = new ArrayList<>();
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_95, null, session1).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_20, null, session1).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_95, null, session2).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_60, null, session2).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_5, null, session2).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_60).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_20, null, session2).getData());
+    handleList.add((QueryHandle) qHelper.executeQuery(COST_5, null, session2).getData());
+
+    List<QueryStatus> statusList = new ArrayList<>();
+    for(QueryHandle handle: handleList){
+      statusList.add(qHelper.getQueryStatus(handle));
+    }
+
+    for(int i=0; i<expectedStatus.length; i++){
+      Assert.assertEquals(statusList.get(i).getStatus(), expectedStatus[i], "failed : query-" + i);
+    }
+  }
+
+
+  @Test(enabled = true)
+  public void queueAndPriorityMaxConcurrentMany() throws Exception {
+
+    long timeToWait= 5 * SECONDS_IN_A_MINUTE; // in seconds
+    int sleepTime = 5, maxConcurrent = 5, queue1Concurrent = 1, queue2Concurrent = 2, priority1 = 2, priority2 = 1;
+
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
+      String.valueOf(maxConcurrent), DriverConfig.PRIORITY_MAX_CONCURRENT,
+      "HIGH=" + String.valueOf(priority1) + ",NORMAL=" + String.valueOf(priority2),
+      DriverConfig.QUEUE_MAX_CONCURRENT,
+      queue1 + "=" + String.valueOf(queue1Concurrent) + "," + queue2 + "=" + String.valueOf(queue2Concurrent));
+
+    Util.changeConfig(map, hiveDriverConf);
+    lens.restart();
+
+    sHelper.setAndValidateParam(session1, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue1);
+    sHelper.setAndValidateParam(session2, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
+
+    List<QueryHandle> handleList = new ArrayList<QueryHandle>();
+    for (int i = 1; i <= 3; i++) {
+      handleList.add((QueryHandle) qHelper.executeQuery(COST_5).getData());
+      handleList.add((QueryHandle) qHelper.executeQuery(COST_20, "", session1).getData());
+      handleList.add((QueryHandle) qHelper.executeQuery(COST_60, "", session2).getData());
+      handleList.add((QueryHandle) qHelper.executeQuery(COST_20).getData());
+      handleList.add((QueryHandle) qHelper.executeQuery(COST_95, "", session1).getData());
+      handleList.add((QueryHandle) qHelper.executeQuery(COST_5, "", session2).getData());
+    }
+
+    List<QueryHandle> running = null, queued = null;
+    for (int t = 0; t < timeToWait; t = t + sleepTime) {
+
+      running = qHelper.getQueryHandleList(null, "RUNNING", "all", sessionHandleString, null, null, hiveDriver);
+      queued = qHelper.getQueryHandleList(null, "QUEUED", "all", sessionHandleString, null, null, hiveDriver);
+      logger.info("Running query count : " + running.size() + "\t Queued query count : " + queued.size());
+
+      if (running.isEmpty() && queued.isEmpty()) {
+        break;
+      }
+
+      Assert.assertTrue(running.size() <= maxConcurrent);
+
+      int pCount1 = 0, pCount2 = 0, queue1Count = 0, queue2Count = 0;
+      for (QueryHandle qh : running) {
+        Priority priority = qHelper.getLensQuery(sessionHandleString, qh).getPriority();
+        String queue = qHelper.getLensQuery(sessionHandleString, qh).getQueryConf().getProperties()
+            .get("mapreduce.job.queuename");
+
+        Assert.assertNotNull(priority);
+        Assert.assertNotNull(queue);
+
+        if (priority.equals(Priority.LOW)){
+          pCount1++;
+        } else if (priority.equals(Priority.VERY_LOW)){
+          pCount2++;
+        }
+
+        if (queue.equals(queue1)){
+          queue1Count++;
+        } else if (queue.equals(queue2)) {
+          queue2Count++;
+        }
+      }
+
+      Assert.assertTrue(pCount1 <= priority1, "proirty-1 count : " + pCount1);
+      Assert.assertTrue(pCount2 <= priority2, "priority-2 count : " + pCount2);
+      Assert.assertTrue(queue1Count <= queue1Concurrent, "queue-1 count : " + queue1Count);
+      Assert.assertTrue(queue2Count <= queue2Concurrent, "queue-2 count : " + queue2Count);
+
+      TimeUnit.SECONDS.sleep(sleepTime);
+    }
+
+    Assert.assertTrue(queued.isEmpty());
+    Assert.assertTrue(running.isEmpty());
+
+    for (QueryHandle q : handleList) {
+      LensQuery lq = qHelper.waitForCompletion(q);
+      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+    }
+  }
+
+  /*
+    LENS-973. Scenario is mentioned in jira
+  */
+
+  @Test(enabled = true)
+  public void queueConstraintFailureOnRestart() throws Exception {
+
+    List<QueryHandle> handleList = new ArrayList<QueryHandle>();
+
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "3",
+        DriverConfig.QUEUE_MAX_CONCURRENT, queue1 + "=1," + queue2 + "=3");
+    Util.changeConfig(map, hiveDriverConf);
+    lens.restart();
+
+    String newSession = sHelper.openSession("user", "pwd", lens.getCurrentDB());
+    sHelper.setAndValidateParam(newSession, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
+    handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.SLEEP_QUERY, null, newSession).getData());
+
+    for(int i=0; i<2; i++){
+      handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_CUBE_QUERY).getData());
+    }
+
+    sHelper.closeSession(newSession);
+    lens.restart();
+    Assert.assertFalse(qHelper.getQueryStatus(handleList.get(0)).finished());
+
+    for(int i=0; i<6; i++){
+      handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_DIM_QUERY).getData());
+    }
+
+    for(QueryHandle handle: handleList){
+      LensQuery lq = qHelper.waitForCompletion(handle);
+      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/throttling/Throttling.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/throttling/Throttling.java b/lens-regression/src/test/java/org/apache/lens/regression/throttling/Throttling.java
deleted file mode 100644
index abf7263..0000000
--- a/lens-regression/src/test/java/org/apache/lens/regression/throttling/Throttling.java
+++ /dev/null
@@ -1,604 +0,0 @@
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.lens.regression.throttling;
-
-import java.lang.reflect.Method;
-import java.util.*;
-import java.util.concurrent.TimeUnit;
-
-import javax.ws.rs.client.WebTarget;
-
-import org.apache.lens.api.Priority;
-import org.apache.lens.api.query.LensQuery;
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryStatus;
-import org.apache.lens.cube.parse.CubeQueryConfUtil;
-import org.apache.lens.driver.hive.HiveDriver;
-import org.apache.lens.regression.core.constants.DriverConfig;
-import org.apache.lens.regression.core.constants.QueryInventory;
-import org.apache.lens.regression.core.helpers.ServiceManagerHelper;
-import org.apache.lens.regression.core.testHelper.BaseTestClass;
-import org.apache.lens.regression.util.Util;
-import org.apache.lens.server.api.LensConfConstants;
-import org.apache.lens.server.api.util.LensUtil;
-
-import org.apache.log4j.Logger;
-
-import org.testng.Assert;
-import org.testng.annotations.*;
-
-public class Throttling extends BaseTestClass {
-
-  WebTarget servLens;
-  String sessionHandleString;
-
-  public static final String SLEEP_QUERY = QueryInventory.getSleepQuery("5");
-  public static final String COST_95 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_95"), "5");
-  public static final String COST_60 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_60"), "5");
-  public static final String COST_30 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_30"), "5");
-  public static final String COST_20 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_20"), "4");
-  public static final String COST_10 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_10"), "4");
-  public static final String COST_5 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_5"), "3");
-  public static final String JDBC_QUERY1 = QueryInventory.getQueryFromInventory("JDBC.QUERY1");
-
-  private static String hiveDriver = "hive/hive1";
-  private final String hiveDriverConf = lens.getServerDir() + "/conf/drivers/hive/hive1/hivedriver-site.xml";
-  private final String backupConfFilePath = lens.getServerDir() + "/conf/drivers/hive/hive1/backup-hivedriver-site.xml";
-
-  private static final long SECONDS_IN_A_MINUTE = 60;
-  private String session1 = null, session2 = null;
-  //TODO : Read queue names from property file
-  private static String queue1 = "dwh", queue2 = "reports";
-
-  private static Logger logger = Logger.getLogger(Throttling.class);
-
-  @BeforeClass(alwaysRun = true)
-  public void initialize() throws Exception {
-    servLens = ServiceManagerHelper.init();
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "10",
-        HiveDriver.HS2_PRIORITY_RANGES, "HIGH,7,NORMAL,30,LOW,90,VERY_LOW");
-    Util.changeConfig(map, hiveDriverConf);
-    lens.restart();
-  }
-
-  @BeforeMethod(alwaysRun = true)
-  public void setUp(Method method) throws Exception {
-    logger.info("Test Name: " + method.getName());
-    Util.runRemoteCommand("cp " + hiveDriverConf + " " + backupConfFilePath);
-
-    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
-    session1 = sHelper.openSession("diff1", "diff1", lens.getCurrentDB());
-    session2 = sHelper.openSession("diff2", "diff2", lens.getCurrentDB());
-    sHelper.setAndValidateParam(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "false");
-    sHelper.setAndValidateParam(session1, CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "false");
-    sHelper.setAndValidateParam(session2, CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "false");
-  }
-
-  @AfterMethod(alwaysRun = true)
-  public void afterMethod(Method method) throws Exception {
-    logger.info("Test Name: " + method.getName());
-    qHelper.killQuery(null, "QUEUED", "all");
-    qHelper.killQuery(null, "RUNNING", "all");
-    qHelper.killQuery(null, "EXECUTED", "all");
-    sHelper.closeSession(session1);
-    sHelper.closeSession(session2);
-    sHelper.closeSession(sessionHandleString);
-
-    Util.runRemoteCommand("cp " + backupConfFilePath + " " + hiveDriverConf);
-  }
-
-  @AfterClass(alwaysRun = false)
-  public void closeSession() throws Exception {
-    lens.restart();
-  }
-
-
-  @Test(enabled = true)
-  public void testHiveThrottling() throws Exception {
-
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "2");
-    Util.changeConfig(map, hiveDriverConf);
-    lens.restart();
-
-    List<QueryHandle> handleList = new ArrayList<>();
-    handleList.add((QueryHandle) qHelper.executeQuery(SLEEP_QUERY).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(SLEEP_QUERY, null, session1).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(SLEEP_QUERY, null, session2).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(SLEEP_QUERY, null).getData());
-
-    Thread.sleep(1000);
-
-    List<QueryStatus> statusList = new ArrayList<>();
-    for(QueryHandle handle : handleList){
-      statusList.add(qHelper.getQueryStatus(handle));
-    }
-
-    Assert.assertEquals(statusList.get(0).getStatus(), QueryStatus.Status.RUNNING);
-    Assert.assertEquals(statusList.get(1).getStatus(), QueryStatus.Status.RUNNING);
-    Assert.assertEquals(statusList.get(2).getStatus(), QueryStatus.Status.QUEUED);
-    Assert.assertEquals(statusList.get(3).getStatus(), QueryStatus.Status.QUEUED);
-
-    qHelper.waitForCompletion(handleList.get(0));
-    Thread.sleep(100);
-    Assert.assertEquals(qHelper.getQueryStatus(handleList.get(2)).getStatus(), QueryStatus.Status.RUNNING);
-  }
-
-
-  @Test(enabled = true)
-  public void testHiveMaxConcurrentRandomQueryIngestion() throws Exception {
-
-    long timeToWait= 5 * SECONDS_IN_A_MINUTE; // in seconds
-    int sleepTime = 3, maxConcurrent = 4;
-    List<QueryHandle> handleList = new ArrayList<QueryHandle>();
-
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
-        String.valueOf(maxConcurrent));
-    Util.changeConfig(map, hiveDriverConf);
-    lens.restart();
-
-    for (int i = 0; i < 5; i++) {
-      handleList.add((QueryHandle) qHelper.executeQuery(SLEEP_QUERY).getData());
-      handleList.add((QueryHandle) qHelper.executeQuery(JDBC_QUERY1).getData());
-      handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_CUBE_QUERY, null, session1).getData());
-    }
-
-    Thread.sleep(50);
-
-    List<QueryHandle> running = null, queued = null;
-    for (int t = 0; t < timeToWait; t = t + sleepTime) {
-
-      running = qHelper.getQueryHandleList(null, "RUNNING", "all", sessionHandleString, null, null, hiveDriver);
-      queued = qHelper.getQueryHandleList(null, "QUEUED", "all", sessionHandleString, null, null, hiveDriver);
-      logger.info("Running query count : " + running.size() + "\t Queued query count : " + queued.size());
-
-      if (running.isEmpty() && queued.isEmpty()) {
-        break;
-      }
-      Assert.assertTrue(running.size() <= maxConcurrent);
-
-      if (t % 30 == 0 && t < 200) {
-        handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_DIM_QUERY).getData());
-        handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.JDBC_CUBE_QUERY).getData());
-        handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_CUBE_QUERY, null, session1).getData());
-      }
-      TimeUnit.SECONDS.sleep(sleepTime);
-    }
-
-    Assert.assertTrue(running.isEmpty());
-    Assert.assertTrue(queued.isEmpty());
-
-    for(QueryHandle q : handleList){
-      LensQuery lq = qHelper.waitForCompletion(q);
-      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-    }
-  }
-
-  @Test(enabled = true)
-  public void testProrityMaxConcurrent() throws Exception {
-
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "5",
-        DriverConfig.PRIORITY_MAX_CONCURRENT, "HIGH=2,VERY_LOW=1", HiveDriver.HS2_PRIORITY_RANGES,
-        "HIGH,7,NORMAL,30,LOW,90,VERY_LOW");
-    Util.changeConfig(map, hiveDriverConf);
-    lens.restart();
-
-    /* First 3 are high priority queries, 2 of them will go to RUNNING, 3rd will be queued as there is a
-      threshold of 2 on HIGH priority. cost_95 queries are very_low priority ones, hence 1st will go to RUNNING,
-      other 1 is queued. cost_20 and cost_60 goes to running as they are normal and low priority queries and there
-      is no limit set for low and normal priority.Last cost_60 query goes to queue as, RUNNING query on this
-      driver has reached max concurrent threshold.
-    */
-
-    String[] queries = {COST_5, COST_5, COST_5, COST_95, COST_95, COST_20, COST_60, COST_60};
-    QueryStatus.Status[] expectedStatus = {QueryStatus.Status.RUNNING, QueryStatus.Status.RUNNING,
-      QueryStatus.Status.QUEUED, QueryStatus.Status.RUNNING, QueryStatus.Status.QUEUED,
-      QueryStatus.Status.RUNNING, QueryStatus.Status.RUNNING, QueryStatus.Status.QUEUED, };
-
-    List<QueryHandle> handleList = new ArrayList<>();
-    for (String query : queries){
-      handleList.add((QueryHandle) qHelper.executeQuery(query).getData());
-    }
-
-    List<QueryStatus.Status> statusList = new ArrayList<>();
-    for (QueryHandle handle : handleList){
-      statusList.add(qHelper.getQueryStatus(handle).getStatus());
-    }
-
-    for (int i=0; i<statusList.size(); i++){
-      Assert.assertEquals(statusList.get(i), expectedStatus[i]);
-    }
-
-    qHelper.waitForCompletion(handleList.get(0));
-    Assert.assertEquals(qHelper.getQueryStatus(handleList.get(2)).getStatus(), QueryStatus.Status.RUNNING);
-  }
-
-  @Test(enabled = true)
-  public void prioritySumMoreThanMaxConcurrent() throws Exception {
-
-    long timeToWait= 5 * SECONDS_IN_A_MINUTE; // in seconds
-    int sleepTime = 5, maxConcurrent = 5, lowConCurrent = 2, veryLowConcurrent = 1, highConcurrent = 4,
-        normalConcurrent = 2;
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
-        String.valueOf(maxConcurrent), DriverConfig.PRIORITY_MAX_CONCURRENT,
-        "LOW=" + String.valueOf(lowConCurrent) + ",VERY_LOW=" + String.valueOf(veryLowConcurrent)
-        + ",NORMAL=" + String.valueOf(normalConcurrent) + ",HIGH=" + String.valueOf(highConcurrent));
-    Util.changeConfig(map, hiveDriverConf);
-    lens.restart();
-
-    List<QueryHandle> handleList = new ArrayList<QueryHandle>();
-    for (int i=1; i<=5; i++){
-      handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
-      handleList.add((QueryHandle) qHelper.executeQuery(COST_60, null, session1).getData());
-      handleList.add((QueryHandle) qHelper.executeQuery(COST_20, null, session2).getData());
-      handleList.add((QueryHandle) qHelper.executeQuery(COST_5).getData());
-    }
-
-    List<QueryHandle> running=null, queued=null;
-    for (int t = 0; t < timeToWait; t = t + sleepTime) {
-
-      running = qHelper.getQueryHandleList(null, "RUNNING", "all", sessionHandleString, null, null, hiveDriver);
-      queued = qHelper.getQueryHandleList(null, "QUEUED", "all", sessionHandleString, null, null, hiveDriver);
-      logger.info("Running query count : " + running.size() + "\t Queued query count : " + queued.size());
-
-      if (running.isEmpty() && queued.isEmpty()) {
-        break;
-      }
-
-      Assert.assertTrue(running.size() <= maxConcurrent);
-
-      int low = 0, veryLow = 0, high = 0, normal = 0;
-      for (QueryHandle qh : running) {
-        Priority p = qHelper.getLensQuery(sessionHandleString, qh).getPriority();
-        Assert.assertNotNull(p);
-        switch (p) {
-        case HIGH:
-          high++;
-          break;
-        case NORMAL:
-          normal++;
-          break;
-        case LOW:
-          low++;
-          break;
-        case VERY_LOW:
-          veryLow++;
-          break;
-        default:
-          throw new Exception("Unexpected Priority");
-        }
-      }
-
-      Assert.assertTrue(low <= lowConCurrent);
-      Assert.assertTrue(veryLow <= veryLowConcurrent);
-      Assert.assertTrue(high <= highConcurrent);
-      Assert.assertTrue(normal <= normalConcurrent);
-
-      TimeUnit.SECONDS.sleep(sleepTime);
-    }
-
-    Assert.assertTrue(queued.isEmpty());
-    Assert.assertTrue(running.isEmpty());
-
-    for (QueryHandle q: handleList){
-      LensQuery lq = qHelper.waitForCompletion(q);
-      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-    }
-  }
-
-
-  @Test(enabled = true)
-  public void queueMaxConcurrent() throws Exception {
-
-    int maxConcurrent = 3, queue1Count = 1, queue2Count = 2;
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
-        String.valueOf(maxConcurrent), DriverConfig.QUEUE_MAX_CONCURRENT,
-        queue1 + "=" + String.valueOf(queue1Count) + "," + queue2 + "=" + String.valueOf(queue2Count));
-    List<QueryHandle> handleList = new ArrayList<>();
-
-    Util.changeConfig(map, hiveDriverConf);
-    lens.restart();
-
-    sHelper.setAndValidateParam(LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue1);
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_60).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
-
-    sHelper.setAndValidateParam(LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_60).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
-
-    List<QueryStatus.Status> statusList = new ArrayList<>();
-    for (QueryHandle handle : handleList){
-      statusList.add(qHelper.getQueryStatus(handle).getStatus());
-    }
-
-    Assert.assertEquals(statusList.get(0), QueryStatus.Status.RUNNING);
-    Assert.assertEquals(statusList.get(1), QueryStatus.Status.QUEUED);
-    Assert.assertEquals(statusList.get(2), QueryStatus.Status.RUNNING);
-    Assert.assertEquals(statusList.get(3), QueryStatus.Status.RUNNING);
-    Assert.assertEquals(statusList.get(4), QueryStatus.Status.QUEUED);
-
-    qHelper.waitForCompletion(handleList.get(0));
-    Assert.assertEquals(qHelper.getQueryStatus(handleList.get(1)).getStatus(), QueryStatus.Status.RUNNING);
-
-    qHelper.waitForCompletion(handleList.get(2));
-    Assert.assertEquals(qHelper.getQueryStatus(handleList.get(4)).getStatus(), QueryStatus.Status.RUNNING);
-  }
-
-  // LENS-1027
-  @Test(enabled = true)
-  public void queueDefaultThresholdConstraint() throws Exception {
-
-    int maxConcurrent = 5, queue1Count = 1, queue2Count = 2;
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
-        String.valueOf(maxConcurrent), DriverConfig.QUEUE_MAX_CONCURRENT,
-        "*=" + String.valueOf(queue1Count) + "," + queue2 + "=" + String.valueOf(queue2Count));
-    List<QueryHandle> handleList = new ArrayList<>();
-
-    Util.changeConfig(map, hiveDriverConf);
-    lens.restart();
-
-    sHelper.setAndValidateParam(LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue1);
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_60).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
-
-    sHelper.setAndValidateParam(LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_60).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_95).getData());
-
-    Thread.sleep(2000);
-
-    List<QueryStatus.Status> statusList = new ArrayList<>();
-    for (QueryHandle handle : handleList){
-      statusList.add(qHelper.getQueryStatus(handle).getStatus());
-    }
-
-    Assert.assertEquals(statusList.get(0), QueryStatus.Status.RUNNING);
-    Assert.assertEquals(statusList.get(1), QueryStatus.Status.QUEUED);
-    Assert.assertEquals(statusList.get(2), QueryStatus.Status.RUNNING);
-    Assert.assertEquals(statusList.get(3), QueryStatus.Status.RUNNING);
-    Assert.assertEquals(statusList.get(4), QueryStatus.Status.QUEUED);
-  }
-
-
-  @Test(enabled = true)
-  public void enableQueueThrottlingWithExistingQueuedQueries() throws Exception {
-
-    long timeToWait= 5 * SECONDS_IN_A_MINUTE; // in seconds
-    int sleepTime = 5, maxConcurrent = 4, queue1Concurrent = 1, queue2Concurrent = 2;
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
-        String.valueOf(maxConcurrent), DriverConfig.QUEUE_MAX_CONCURRENT,
-        queue1 + "=" + String.valueOf(queue1Concurrent) + "," + queue2 + "=" + String.valueOf(queue2Concurrent));
-
-    sHelper.setAndValidateParam(session1, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue1);
-    sHelper.setAndValidateParam(session2, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
-
-    List<QueryHandle> handleList = new ArrayList<QueryHandle>();
-    for (int i = 1; i <= 3; i++) {
-      handleList.add((QueryHandle)qHelper.executeQuery(COST_95).getData());
-      handleList.add((QueryHandle)qHelper.executeQuery(COST_20, "", session1).getData());
-      handleList.add((QueryHandle)qHelper.executeQuery(COST_95, "", session2).getData());
-    }
-
-    Util.changeConfig(map, hiveDriverConf);
-    lens.restart();
-
-    for (int i = 1; i <= 2; i++) {
-      handleList.add((QueryHandle)qHelper.executeQuery(COST_95).getData());
-      handleList.add((QueryHandle)qHelper.executeQuery(COST_20, "", session1).getData());
-      handleList.add((QueryHandle)qHelper.executeQuery(COST_95, "", session2).getData());
-    }
-
-    List<QueryHandle> running = null, queued = null;
-    for (int t = 0; t < timeToWait; t = t + sleepTime) {
-
-      running = qHelper.getQueryHandleList(null, "RUNNING", "all", sessionHandleString, null, null, hiveDriver);
-      queued = qHelper.getQueryHandleList(null, "QUEUED", "all", sessionHandleString,  null, null, hiveDriver);
-      logger.info("Running query count : " + running.size() + "\t Queued query count : " + queued.size());
-
-      if (running.isEmpty() && queued.isEmpty()) {
-        break;
-      }
-      Assert.assertTrue(running.size() <= maxConcurrent);
-
-      int queue1Count = 0, queue2Count = 0;
-      for (QueryHandle qh : running) {
-        String queue = qHelper.getLensQuery(sessionHandleString, qh).getQueryConf().getProperties()
-            .get("mapreduce.job.queuename");
-        Assert.assertNotNull(queue);
-
-        if (queue.equals(queue1)) {
-          queue1Count++;
-        } else if (queue.equals(queue2)) {
-          queue2Count++;
-        }
-      }
-
-      Assert.assertTrue(queue1Count <= queue1Concurrent, "queue1 count : " + queue1Count);
-      Assert.assertTrue(queue2Count <= queue2Concurrent, "queue2 count : " + queue2Count);
-      TimeUnit.SECONDS.sleep(sleepTime);
-    }
-
-    Assert.assertTrue(running.isEmpty());
-    Assert.assertTrue(queued.isEmpty());
-
-    for(QueryHandle q: handleList){
-      LensQuery lq = qHelper.waitForCompletion(q);
-      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-    }
-  }
-
-
-  @Test(enabled = true)
-  public void queueAndPriorityMaxConcurrent() throws Exception {
-
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "5",
-      DriverConfig.PRIORITY_MAX_CONCURRENT, "LOW=2,VERY_LOW=1",
-      DriverConfig.QUEUE_MAX_CONCURRENT, queue1 + "=1," + queue2 + "=2");
-
-    Util.changeConfig(map, hiveDriverConf);
-    lens.restart();
-
-    sHelper.setAndValidateParam(session1, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue1);
-    sHelper.setAndValidateParam(session2, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
-
-    QueryStatus.Status[] expectedStatus = {QueryStatus.Status.RUNNING, QueryStatus.Status.QUEUED,
-      QueryStatus.Status.QUEUED, QueryStatus.Status.RUNNING, QueryStatus.Status.RUNNING,
-      QueryStatus.Status.RUNNING, QueryStatus.Status.QUEUED, QueryStatus.Status.QUEUED, };
-
-    List<QueryHandle> handleList = new ArrayList<>();
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_95, null, session1).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_20, null, session1).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_95, null, session2).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_60, null, session2).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_5, null, session2).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_60).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_20, null, session2).getData());
-    handleList.add((QueryHandle) qHelper.executeQuery(COST_5, null, session2).getData());
-
-    List<QueryStatus> statusList = new ArrayList<>();
-    for(QueryHandle handle: handleList){
-      statusList.add(qHelper.getQueryStatus(handle));
-    }
-
-    for(int i=0; i<expectedStatus.length; i++){
-      Assert.assertEquals(statusList.get(i).getStatus(), expectedStatus[i], "failed : query-" + i);
-    }
-  }
-
-
-  @Test(enabled = true)
-  public void queueAndPriorityMaxConcurrentMany() throws Exception {
-
-    long timeToWait= 5 * SECONDS_IN_A_MINUTE; // in seconds
-    int sleepTime = 5, maxConcurrent = 5, queue1Concurrent = 1, queue2Concurrent = 2, priority1 = 2, priority2 = 1;
-
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES,
-      String.valueOf(maxConcurrent), DriverConfig.PRIORITY_MAX_CONCURRENT,
-      "HIGH=" + String.valueOf(priority1) + ",NORMAL=" + String.valueOf(priority2),
-      DriverConfig.QUEUE_MAX_CONCURRENT,
-      queue1 + "=" + String.valueOf(queue1Concurrent) + "," + queue2 + "=" + String.valueOf(queue2Concurrent));
-
-    Util.changeConfig(map, hiveDriverConf);
-    lens.restart();
-
-    sHelper.setAndValidateParam(session1, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue1);
-    sHelper.setAndValidateParam(session2, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
-
-    List<QueryHandle> handleList = new ArrayList<QueryHandle>();
-    for (int i = 1; i <= 3; i++) {
-      handleList.add((QueryHandle) qHelper.executeQuery(COST_5).getData());
-      handleList.add((QueryHandle) qHelper.executeQuery(COST_20, "", session1).getData());
-      handleList.add((QueryHandle) qHelper.executeQuery(COST_60, "", session2).getData());
-      handleList.add((QueryHandle) qHelper.executeQuery(COST_20).getData());
-      handleList.add((QueryHandle) qHelper.executeQuery(COST_95, "", session1).getData());
-      handleList.add((QueryHandle) qHelper.executeQuery(COST_5, "", session2).getData());
-    }
-
-    List<QueryHandle> running = null, queued = null;
-    for (int t = 0; t < timeToWait; t = t + sleepTime) {
-
-      running = qHelper.getQueryHandleList(null, "RUNNING", "all", sessionHandleString, null, null, hiveDriver);
-      queued = qHelper.getQueryHandleList(null, "QUEUED", "all", sessionHandleString, null, null, hiveDriver);
-      logger.info("Running query count : " + running.size() + "\t Queued query count : " + queued.size());
-
-      if (running.isEmpty() && queued.isEmpty()) {
-        break;
-      }
-
-      Assert.assertTrue(running.size() <= maxConcurrent);
-
-      int pCount1 = 0, pCount2 = 0, queue1Count = 0, queue2Count = 0;
-      for (QueryHandle qh : running) {
-        Priority priority = qHelper.getLensQuery(sessionHandleString, qh).getPriority();
-        String queue = qHelper.getLensQuery(sessionHandleString, qh).getQueryConf().getProperties()
-            .get("mapreduce.job.queuename");
-
-        Assert.assertNotNull(priority);
-        Assert.assertNotNull(queue);
-
-        if (priority.equals(Priority.LOW)){
-          pCount1++;
-        } else if (priority.equals(Priority.VERY_LOW)){
-          pCount2++;
-        }
-
-        if (queue.equals(queue1)){
-          queue1Count++;
-        } else if (queue.equals(queue2)) {
-          queue2Count++;
-        }
-      }
-
-      Assert.assertTrue(pCount1 <= priority1, "proirty-1 count : " + pCount1);
-      Assert.assertTrue(pCount2 <= priority2, "priority-2 count : " + pCount2);
-      Assert.assertTrue(queue1Count <= queue1Concurrent, "queue-1 count : " + queue1Count);
-      Assert.assertTrue(queue2Count <= queue2Concurrent, "queue-2 count : " + queue2Count);
-
-      TimeUnit.SECONDS.sleep(sleepTime);
-    }
-
-    Assert.assertTrue(queued.isEmpty());
-    Assert.assertTrue(running.isEmpty());
-
-    for (QueryHandle q : handleList) {
-      LensQuery lq = qHelper.waitForCompletion(q);
-      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-    }
-  }
-
-  /*
-    LENS-973. Scenario is mentioned in jira
-  */
-
-  @Test(enabled = true)
-  public void queueConstraintFailureOnRestart() throws Exception {
-
-    List<QueryHandle> handleList = new ArrayList<QueryHandle>();
-
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "3",
-        DriverConfig.QUEUE_MAX_CONCURRENT, queue1 + "=1," + queue2 + "=3");
-    Util.changeConfig(map, hiveDriverConf);
-    lens.restart();
-
-    String newSession = sHelper.openSession("user", "pwd", lens.getCurrentDB());
-    sHelper.setAndValidateParam(newSession, LensConfConstants.MAPRED_JOB_QUEUE_NAME, queue2);
-    handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.SLEEP_QUERY, null, newSession).getData());
-
-    for(int i=0; i<2; i++){
-      handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_CUBE_QUERY).getData());
-    }
-
-    sHelper.closeSession(newSession);
-    lens.restart();
-    Assert.assertFalse(qHelper.getQueryStatus(handleList.get(0)).finished());
-
-    for(int i=0; i<6; i++){
-      handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_DIM_QUERY).getData());
-    }
-
-    for(QueryHandle handle: handleList){
-      LensQuery lq = qHelper.waitForCompletion(handle);
-      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-    }
-  }
-}


[02/16] lens git commit: LENS-1378 : Fix dimensions to query from DenormalizationResolver for expression fields

Posted by pr...@apache.org.
LENS-1378 : Fix dimensions to query from DenormalizationResolver for expression fields


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

Branch: refs/heads/lens-1381
Commit: 4fb2506193acd3efa7d90bb640bd50e2aefe5d6a
Parents: 98990c3
Author: Amareshwari Sriramadasu <am...@apache.org>
Authored: Wed Jan 18 17:46:58 2017 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Wed Jan 18 17:46:58 2017 +0530

----------------------------------------------------------------------
 .../lens/cube/parse/CandidateTableResolver.java |   2 +-
 .../lens/cube/parse/CubeQueryContext.java       |  11 +-
 .../cube/parse/DenormalizationResolver.java     | 167 +++++++++++++------
 .../lens/cube/parse/ExpressionResolver.java     | 107 ++++++------
 .../lens/cube/parse/QueriedPhraseContext.java   |   2 +-
 .../lens/cube/parse/TimeRangeChecker.java       |  18 --
 .../lens/cube/parse/TrackDenormContext.java     |  37 ++++
 .../apache/lens/cube/parse/CubeTestSetup.java   |  11 +-
 .../lens/cube/parse/TestExpressionResolver.java |  15 ++
 .../lens/cube/parse/TestJoinResolver.java       |   2 +-
 tools/conf/server/logback.xml                   |   4 +-
 11 files changed, 248 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/4fb25061/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
index e7fc557..e9270ea 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
@@ -697,7 +697,7 @@ class CandidateTableResolver implements ContextRewriter {
                     i.remove();
                     break;
                   }
-                } else if (!cubeql.getDeNormCtx().addRefUsage(cdim, col, dim.getName())) {
+                } else if (!cubeql.getDeNormCtx().addRefUsage(cubeql, cdim, col, dim.getName())) {
                   // check if it available as reference, if not remove the
                   // candidate
                   log.info("Not considering dimtable: {} as column {} is not available", cdim, col);

http://git-wip-us.apache.org/repos/asf/lens/blob/4fb25061/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java
index e83ae76..125b432 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java
@@ -59,7 +59,7 @@ import lombok.*;
 import lombok.extern.slf4j.Slf4j;
 
 @Slf4j
-public class CubeQueryContext extends TracksQueriedColumns implements QueryAST {
+public class CubeQueryContext extends TracksQueriedColumns implements QueryAST, TrackDenormContext {
   public static final String TIME_RANGE_FUNC = "time_range_in";
   public static final String NOW = "now";
   public static final String DEFAULT_TABLE = "_default_";
@@ -922,7 +922,8 @@ public class CubeQueryContext extends TracksQueriedColumns implements QueryAST {
     Set<Dimension> exprDimensions = new HashSet<>();
     if (cfacts != null) {
       for (CandidateFact cfact : cfacts) {
-        Set<Dimension> factExprDimTables = exprCtx.rewriteExprCtx(cfact, dimsToQuery, cfacts.size() > 1 ? cfact : this);
+        Set<Dimension> factExprDimTables = exprCtx.rewriteExprCtx(this, cfact, dimsToQuery,
+          cfacts.size() > 1 ? cfact : this);
         exprDimensions.addAll(factExprDimTables);
         if (cfacts.size() > 1) {
           factDimMap.get(cfact).addAll(factExprDimTables);
@@ -933,7 +934,7 @@ public class CubeQueryContext extends TracksQueriedColumns implements QueryAST {
       }
     } else {
       // dim only query
-      exprDimensions.addAll(exprCtx.rewriteExprCtx(null, dimsToQuery, this));
+      exprDimensions.addAll(exprCtx.rewriteExprCtx(this, null, dimsToQuery, this));
     }
     dimsToQuery.putAll(pickCandidateDimsToQuery(exprDimensions));
     log.info("facts:{}, dimsToQuery: {}", cfacts, dimsToQuery);
@@ -942,14 +943,14 @@ public class CubeQueryContext extends TracksQueriedColumns implements QueryAST {
     Set<Dimension> denormTables = new HashSet<>();
     if (cfacts != null) {
       for (CandidateFact cfact : cfacts) {
-        Set<Dimension> factDenormTables = deNormCtx.rewriteDenormctx(cfact, dimsToQuery, cfacts.size() > 1);
+        Set<Dimension> factDenormTables = deNormCtx.rewriteDenormctx(this, cfact, dimsToQuery, cfacts.size() > 1);
         denormTables.addAll(factDenormTables);
         if (cfacts.size() > 1) {
           factDimMap.get(cfact).addAll(factDenormTables);
         }
       }
     } else {
-      denormTables.addAll(deNormCtx.rewriteDenormctx(null, dimsToQuery, false));
+      denormTables.addAll(deNormCtx.rewriteDenormctx(this, null, dimsToQuery, false));
     }
     dimsToQuery.putAll(pickCandidateDimsToQuery(denormTables));
     log.info("facts:{}, dimsToQuery: {}", cfacts, dimsToQuery);

http://git-wip-us.apache.org/repos/asf/lens/blob/4fb25061/lens-cube/src/main/java/org/apache/lens/cube/parse/DenormalizationResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/DenormalizationResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/DenormalizationResolver.java
index 40ed387..cb26878 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/DenormalizationResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/DenormalizationResolver.java
@@ -87,23 +87,19 @@ public class DenormalizationResolver implements ContextRewriter {
 
   public static class DenormalizationContext {
     // map of column name to all references
+    @Getter
     private Map<String, Set<ReferencedQueriedColumn>> referencedCols = new HashMap<>();
 
     // candidate table name to all the references columns it needs
+    @Getter
     private Map<String, Set<ReferencedQueriedColumn>> tableToRefCols = new HashMap<>();
 
-    private CubeQueryContext cubeql;
-
     // set of all picked references once all candidate tables are picked
     private Set<PickedReference> pickedRefs = new HashSet<>();
     // index on column name for picked references with map from column name to
     // pickedrefs
     private Map<String, Set<PickedReference>> pickedReferences = new HashMap<>();
 
-    DenormalizationContext(CubeQueryContext cubeql) {
-      this.cubeql = cubeql;
-    }
-
     void addReferencedCol(String col, ReferencedQueriedColumn refer) {
       Set<ReferencedQueriedColumn> refCols = referencedCols.get(col);
       if (refCols == null) {
@@ -116,7 +112,7 @@ public class DenormalizationResolver implements ContextRewriter {
     // When candidate table does not have the field, this method checks
     // if the field can be reached through reference,
     // if yes adds the ref usage and returns to true, if not returns false.
-    boolean addRefUsage(CandidateTable table, String col, String srcTbl) throws LensException {
+    boolean addRefUsage(CubeQueryContext cubeql, CandidateTable table, String col, String srcTbl) throws LensException {
       // available as referenced col
       if (referencedCols.containsKey(col)) {
         for (ReferencedQueriedColumn refer : referencedCols.get(col)) {
@@ -144,10 +140,6 @@ public class DenormalizationResolver implements ContextRewriter {
       return false;
     }
 
-    Map<String, Set<ReferencedQueriedColumn>> getReferencedCols() {
-      return referencedCols;
-    }
-
     private void addPickedReference(String col, PickedReference refer) {
       Set<PickedReference> refCols = pickedReferences.get(col);
       if (refCols == null) {
@@ -169,23 +161,24 @@ public class DenormalizationResolver implements ContextRewriter {
       return null;
     }
 
-    public Set<Dimension> rewriteDenormctx(CandidateFact cfact, Map<Dimension, CandidateDim> dimsToQuery,
-      boolean replaceFact) throws LensException {
+    public Set<Dimension> rewriteDenormctx(CubeQueryContext cubeql, CandidateFact cfact, Map<Dimension,
+      CandidateDim> dimsToQuery, boolean replaceFact) throws LensException {
       Set<Dimension> refTbls = new HashSet<>();
+      log.info("Doing denorm changes for fact :{}", cfact);
 
       if (!tableToRefCols.isEmpty()) {
         // pick referenced columns for fact
         if (cfact != null) {
-          pickColumnsForTable(cfact.getName());
+          pickColumnsForTable(cubeql, cfact.getName());
         }
         // pick referenced columns for dimensions
         if (dimsToQuery != null && !dimsToQuery.isEmpty()) {
           for (CandidateDim cdim : dimsToQuery.values()) {
-            pickColumnsForTable(cdim.getName());
+            pickColumnsForTable(cubeql, cdim.getName());
           }
         }
         // Replace picked reference in all the base trees
-        replaceReferencedColumns(cfact, replaceFact);
+        replaceReferencedColumns(cubeql, cfact, replaceFact);
 
         // Add the picked references to dimsToQuery
         for (PickedReference picked : pickedRefs) {
@@ -195,9 +188,45 @@ public class DenormalizationResolver implements ContextRewriter {
           }
         }
       }
+      pickedReferences.clear();
+      pickedRefs.clear();
       return refTbls;
     }
 
+    public boolean hasReferences() {
+      return !tableToRefCols.isEmpty();
+    }
+    public Set<Dimension> rewriteDenormctxInExpression(CubeQueryContext cubeql, CandidateFact cfact, Map<Dimension,
+      CandidateDim> dimsToQuery, ASTNode exprAST) throws LensException {
+      Set<Dimension> refTbls = new HashSet<>();
+
+      log.info("Doing denorm changes for expressions in fact :{}", cfact);
+      if (!tableToRefCols.isEmpty()) {
+        // pick referenced columns for fact
+        if (cfact != null) {
+          pickColumnsForTable(cubeql, cfact.getName());
+        }
+        // pick referenced columns for dimensions
+        if (dimsToQuery != null && !dimsToQuery.isEmpty()) {
+          for (CandidateDim cdim : dimsToQuery.values()) {
+            pickColumnsForTable(cubeql, cdim.getName());
+          }
+        }
+        // Replace picked reference in expression ast
+        resolveClause(exprAST);
+
+        // Add the picked references to dimsToQuery
+        for (PickedReference picked : pickedRefs) {
+          if (isPickedFor(picked, cfact, dimsToQuery)) {
+            refTbls.add((Dimension) cubeql.getCubeTableForAlias(picked.getChainRef().getChainName()));
+            cubeql.addColumnsQueried(picked.getChainRef().getChainName(), picked.getChainRef().getRefColumn());
+          }
+        }
+      }
+      pickedReferences.clear();
+      pickedRefs.clear();
+      return refTbls;
+    }
     // checks if the reference if picked for facts and dimsToQuery passed
     private boolean isPickedFor(PickedReference picked, CandidateFact cfact, Map<Dimension, CandidateDim> dimsToQuery) {
       if (cfact != null && picked.pickedFor.equalsIgnoreCase(cfact.getName())) {
@@ -213,7 +242,7 @@ public class DenormalizationResolver implements ContextRewriter {
       return false;
     }
 
-    private void pickColumnsForTable(String tbl) throws LensException {
+    private void pickColumnsForTable(CubeQueryContext cubeql, String tbl) throws LensException {
       if (tableToRefCols.containsKey(tbl)) {
         for (ReferencedQueriedColumn refered : tableToRefCols.get(tbl)) {
           Iterator<ChainRefCol> iter = refered.chainRefCols.iterator();
@@ -237,27 +266,57 @@ public class DenormalizationResolver implements ContextRewriter {
       }
     }
 
-    private void replaceReferencedColumns(CandidateFact cfact, boolean replaceFact) throws LensException {
+    public void pruneReferences(CubeQueryContext cubeql) {
+      for (Set<ReferencedQueriedColumn> referencedQueriedColumns : referencedCols.values()) {
+        for(Iterator<ReferencedQueriedColumn> iterator = referencedQueriedColumns.iterator(); iterator.hasNext();) {
+          ReferencedQueriedColumn rqc = iterator.next();
+          for (Iterator<ChainRefCol> iter = rqc.chainRefCols.iterator(); iter.hasNext();) {
+            // remove unreachable references
+            ChainRefCol reference = iter.next();
+            if (cubeql.getAutoJoinCtx() == null || !cubeql.getAutoJoinCtx().isReachableDim(
+              (Dimension) cubeql.getCubeTableForAlias(reference.getChainName()), reference.getChainName())) {
+              log.info("{} is not reachable", reference.getChainName());
+              iter.remove();
+            }
+          }
+          if (rqc.chainRefCols.isEmpty()) {
+            log.info("The referenced column: {} is not reachable", rqc.col.getName());
+            iterator.remove();
+            continue;
+          }
+          // do column life validation
+          for (TimeRange range : cubeql.getTimeRanges()) {
+            if (!rqc.col.isColumnAvailableInTimeRange(range)) {
+              log.info("The referenced column: {} is not in the range queried", rqc.col.getName());
+              iterator.remove();
+              break;
+            }
+          }
+        }
+      }
+    }
+    private void replaceReferencedColumns(CubeQueryContext cubeql, CandidateFact cfact, boolean replaceFact)
+      throws LensException {
       QueryAST ast = cubeql;
       boolean factRefExists = cfact != null && tableToRefCols.get(cfact.getName()) != null && !tableToRefCols.get(cfact
         .getName()).isEmpty();
       if (replaceFact && factRefExists) {
         ast = cfact;
       }
-      resolveClause(cubeql, ast.getSelectAST());
+      resolveClause(ast.getSelectAST());
       if (factRefExists) {
         for (ASTNode storageWhereClauseAST : cfact.getStorgeWhereClauseMap().values()) {
-          resolveClause(cubeql, storageWhereClauseAST);
+          resolveClause(storageWhereClauseAST);
         }
       } else {
-        resolveClause(cubeql, ast.getWhereAST());
+        resolveClause(ast.getWhereAST());
       }
-      resolveClause(cubeql, ast.getGroupByAST());
-      resolveClause(cubeql, ast.getHavingAST());
-      resolveClause(cubeql, cubeql.getOrderByAST());
+      resolveClause(ast.getGroupByAST());
+      resolveClause(ast.getHavingAST());
+      resolveClause(cubeql.getOrderByAST());
     }
 
-    private void resolveClause(CubeQueryContext query, ASTNode node) throws LensException {
+    private void resolveClause(ASTNode node) throws LensException {
       if (node == null) {
         return;
       }
@@ -288,9 +347,22 @@ public class DenormalizationResolver implements ContextRewriter {
         // recurse down
         for (int i = 0; i < node.getChildCount(); i++) {
           ASTNode child = (ASTNode) node.getChild(i);
-          resolveClause(query, child);
+          resolveClause(child);
+        }
+      }
+    }
+
+    public Set<String> getNonReachableReferenceFields(String table) {
+      Set<String> nonReachableFields = new HashSet<>();
+      if (tableToRefCols.containsKey(table)) {
+        for (ReferencedQueriedColumn refcol : tableToRefCols.get(table)) {
+          if (getReferencedCols().get(refcol.col.getName()).isEmpty()) {
+            log.info("For table:{}, the column {} is not available", table, refcol.col);
+            nonReachableFields.add(refcol.col.getName());
+          }
         }
       }
+      return nonReachableFields;
     }
   }
 
@@ -320,6 +392,14 @@ public class DenormalizationResolver implements ContextRewriter {
       }
     }
   }
+  private static DenormalizationContext getOrCreateDeNormCtx(TrackDenormContext tdc) {
+    DenormalizationContext denormCtx = tdc.getDeNormCtx();
+    if (denormCtx == null) {
+      denormCtx = new DenormalizationContext();
+      tdc.setDeNormCtx(denormCtx);
+    }
+    return denormCtx;
+  }
   /**
    * Find all de-normalized columns, if these columns are not directly available in candidate tables, query will be
    * replaced with the corresponding table reference
@@ -329,33 +409,29 @@ public class DenormalizationResolver implements ContextRewriter {
     DenormalizationContext denormCtx = cubeql.getDeNormCtx();
     if (denormCtx == null) {
       // Adds all the reference dimensions as eligible for denorm fields
-      denormCtx = new DenormalizationContext(cubeql);
-      cubeql.setDeNormCtx(denormCtx);
       // add ref columns in cube
-      addRefColsQueried(cubeql, cubeql, denormCtx);
+      addRefColsQueried(cubeql, cubeql, getOrCreateDeNormCtx(cubeql));
       // add ref columns from expressions
       for (Set<ExpressionContext> ecSet : cubeql.getExprCtx().getAllExprsQueried().values()) {
         for (ExpressionContext ec : ecSet) {
           for (ExprSpecContext esc : ec.getAllExprs()) {
-            addRefColsQueried(cubeql, esc, denormCtx);
+            addRefColsQueried(cubeql, esc, getOrCreateDeNormCtx(esc));
           }
         }
       }
     } else if (!denormCtx.tableToRefCols.isEmpty()) {
+      denormCtx.pruneReferences(cubeql);
       // In the second iteration of denorm resolver
       // candidate tables which require denorm fields and the refernces are no
       // more valid will be pruned
       if (cubeql.getCube() != null && !cubeql.getCandidateFacts().isEmpty()) {
         for (Iterator<CandidateFact> i = cubeql.getCandidateFacts().iterator(); i.hasNext();) {
           CandidateFact cfact = i.next();
-          if (denormCtx.tableToRefCols.containsKey(cfact.getName())) {
-            for (ReferencedQueriedColumn refcol : denormCtx.tableToRefCols.get(cfact.getName())) {
-              if (denormCtx.getReferencedCols().get(refcol.col.getName()).isEmpty()) {
-                log.info("Not considering fact table:{} as column {} is not available", cfact, refcol.col);
-                cubeql.addFactPruningMsgs(cfact.fact, CandidateTablePruneCause.columnNotFound(refcol.col.getName()));
-                i.remove();
-              }
-            }
+          Set<String> nonReachableFields = denormCtx.getNonReachableReferenceFields(cfact.getName());
+          if (!nonReachableFields.isEmpty()) {
+            log.info("Not considering fact table:{} as columns {} are not available", cfact, nonReachableFields);
+            cubeql.addFactPruningMsgs(cfact.fact, CandidateTablePruneCause.columnNotFound(nonReachableFields));
+            i.remove();
           }
         }
         if (cubeql.getCandidateFacts().size() == 0) {
@@ -368,15 +444,12 @@ public class DenormalizationResolver implements ContextRewriter {
         for (Dimension dim : cubeql.getDimensions()) {
           for (Iterator<CandidateDim> i = cubeql.getCandidateDimTables().get(dim).iterator(); i.hasNext();) {
             CandidateDim cdim = i.next();
-            if (denormCtx.tableToRefCols.containsKey(cdim.getName())) {
-              for (ReferencedQueriedColumn refcol : denormCtx.tableToRefCols.get(cdim.getName())) {
-                if (denormCtx.getReferencedCols().get(refcol.col.getName()).isEmpty()) {
-                  log.info("Not considering dim table:{} as column {} is not available", cdim, refcol.col);
-                  cubeql.addDimPruningMsgs(dim, cdim.dimtable,
-                    CandidateTablePruneCause.columnNotFound(refcol.col.getName()));
-                  i.remove();
-                }
-              }
+            Set<String> nonReachableFields = denormCtx.getNonReachableReferenceFields(cdim.getName());
+            if (!nonReachableFields.isEmpty()) {
+              log.info("Not considering dim table:{} as column {} is not available", cdim, nonReachableFields);
+              cubeql.addDimPruningMsgs(dim, cdim.dimtable,
+                CandidateTablePruneCause.columnNotFound(nonReachableFields));
+              i.remove();
             }
           }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/4fb25061/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
index 60dacdb..0ea0b1c 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
@@ -154,7 +154,7 @@ class ExpressionResolver implements ContextRewriter {
     void addEvaluable(CubeQueryContext cubeql, CandidateTable cTable, ExprSpecContext esc) throws LensException {
       Set<ExprSpecContext> evalSet = evaluableExpressions.get(cTable);
       if (evalSet == null) {
-        evalSet = new LinkedHashSet<ExprSpecContext>();
+        evalSet = new LinkedHashSet<>();
         evaluableExpressions.put(cTable, evalSet);
       }
       // add optional dimensions involved in expressions
@@ -196,13 +196,16 @@ class ExpressionResolver implements ContextRewriter {
     }
   }
 
-  static class ExprSpecContext extends TracksQueriedColumns {
+  static class ExprSpecContext extends TracksQueriedColumns implements TrackDenormContext {
     private Set<ExprSpec> exprSpecs = new LinkedHashSet<>();
     @Getter
     @Setter
     private ASTNode finalAST;
     @Getter
     private Set<Dimension> exprDims = new HashSet<>();
+    @Getter
+    @Setter
+    private DenormalizationResolver.DenormalizationContext deNormCtx;
 
     ExprSpecContext(ExprSpec exprSpec, CubeQueryContext cubeql) throws LensException {
       // replaces table names in expression with aliases in the query
@@ -221,6 +224,7 @@ class ExpressionResolver implements ContextRewriter {
       finalAST = AliasReplacer.replaceAliases(finalAST, 0, cubeql.getColToTableAlias());
     }
 
+
     void resolveColumns(CubeQueryContext cubeql) throws LensException {
       // finds all columns and table aliases in the expression
       ColumnResolver.getColsForTree(cubeql, finalAST, this, false);
@@ -267,13 +271,31 @@ class ExpressionResolver implements ContextRewriter {
     public String toString() {
       return HQLParser.getString(finalAST);
     }
+
   }
 
-  @AllArgsConstructor
+  @RequiredArgsConstructor
   @ToString
   private static class PickedExpression {
-    private String srcAlias;
-    private ExprSpecContext pickedCtx;
+    private final String srcAlias;
+    private final ExprSpecContext pickedCtx;
+    private transient ASTNode reWrittenAST = null;
+
+    /*
+    Initialized rewrittenAST as copy of final AST if boolean is passed. Copy would be required if finalAST gets
+    modified because of denormalization context.
+    Otherwise, it is final AST reference, without any copy.
+     */
+    void initRewrittenAST(boolean copyFinal) {
+      if (copyFinal) {
+        reWrittenAST = MetastoreUtil.copyAST(pickedCtx.getFinalAST());
+      } else {
+        reWrittenAST = pickedCtx.getFinalAST();
+      }
+    }
+    ASTNode getRewrittenAST() {
+      return reWrittenAST;
+    }
   }
 
   static class ExpressionResolverContext {
@@ -357,7 +379,7 @@ class ExpressionResolver implements ContextRewriter {
           boolean isEvaluable = true;
           for (String col : columns) {
             if (!cTable.getColumns().contains(col.toLowerCase())) {
-              if (!cubeql.getDeNormCtx().addRefUsage(cTable, col, cTable.getBaseTable().getName())) {
+              if (!esc.getDeNormCtx().addRefUsage(cubeql, cTable, col, cTable.getBaseTable().getName())) {
                 // check if it is available as reference, if not expression is not evaluable
                 log.debug("{} = {} is not evaluable in {}", expr, esc, cTable);
                 isEvaluable = false;
@@ -379,49 +401,10 @@ class ExpressionResolver implements ContextRewriter {
       return ec.isEvaluable(cTable);
     }
 
-    /**
-     *
-     * @param exprs
-     * @return
-     */
-    public boolean allNotEvaluable(Set<String> exprs, CandidateTable cTable) {
-      for (String expr : exprs) {
-        if (isEvaluable(expr, cTable)) {
-          return false;
-        }
-      }
-      return true;
-    }
-
-    public Collection<String> coveringExpressions(Set<String> exprs, CandidateTable cTable) {
-      Set<String> coveringSet = new HashSet<String>();
-      for (String expr : exprs) {
-        if (isEvaluable(expr, cTable)) {
-          coveringSet.add(expr);
-        }
-      }
-      return coveringSet;
-    }
-
-    /**
-     * Returns true if all passed expressions are evaluable
-     *
-     * @param cTable
-     * @param exprs
-     * @return
-     */
-    public boolean allEvaluable(CandidateTable cTable, Set<String> exprs) {
-      for (String expr : exprs) {
-        if (!isEvaluable(expr, cTable)) {
-          return false;
-        }
-      }
-      return true;
-    }
-
-    public Set<Dimension> rewriteExprCtx(CandidateFact cfact, Map<Dimension, CandidateDim> dimsToQuery,
-      QueryAST queryAST) throws LensException {
+    public Set<Dimension> rewriteExprCtx(CubeQueryContext cubeql, CandidateFact cfact, Map<Dimension,
+      CandidateDim> dimsToQuery, QueryAST queryAST) throws LensException {
       Set<Dimension> exprDims = new HashSet<Dimension>();
+      log.info("Picking expressions for fact {} ", cfact);
       if (!allExprsQueried.isEmpty()) {
         // pick expressions for fact
         if (cfact != null) {
@@ -433,16 +416,21 @@ class ExpressionResolver implements ContextRewriter {
             pickExpressionsForTable(cdim);
           }
         }
-        // Replace picked expressions in all the base trees
-        replacePickedExpressions(cfact, queryAST);
-        log.debug("Picked expressions: {}", pickedExpressions);
+        log.info("Picked expressions: {}", pickedExpressions);
         for (Set<PickedExpression> peSet : pickedExpressions.values()) {
           for (PickedExpression pe : peSet) {
             exprDims.addAll(pe.pickedCtx.exprDims);
+            pe.initRewrittenAST(pe.pickedCtx.deNormCtx.hasReferences());
+            exprDims.addAll(pe.pickedCtx.deNormCtx.rewriteDenormctxInExpression(cubeql, cfact, dimsToQuery,
+              pe.getRewrittenAST()));
           }
         }
+        // Replace picked expressions in all the base trees
+        replacePickedExpressions(cfact, queryAST);
       }
+
       pickedExpressions.clear();
+
       return exprDims;
     }
 
@@ -488,7 +476,7 @@ class ExpressionResolver implements ContextRewriter {
               if (pickedExpressions.containsKey(column)) {
                 PickedExpression expr = getPickedExpression(column, tabident.getText().toLowerCase());
                 if (expr != null) {
-                  node.setChild(i, replaceAlias(expr.pickedCtx.finalAST, cubeql));
+                  node.setChild(i, replaceAlias(expr.getRewrittenAST(), cubeql));
                 }
               }
             }
@@ -552,6 +540,21 @@ class ExpressionResolver implements ContextRewriter {
             if (removed) {
               continue;
             }
+            // Remove expressions for which denormalized columns are no more reachable
+            esc.getDeNormCtx().pruneReferences(cubeql);
+            for (String table : esc.getDeNormCtx().getTableToRefCols().keySet()) {
+              Set<String> nonReachableFields = esc.getDeNormCtx().getNonReachableReferenceFields(table);
+              if (!nonReachableFields.isEmpty()) {
+                log.info("Removing expression {} as columns {} are not available", esc, nonReachableFields);
+                iterator.remove();
+                removedEsc.add(esc);
+                removed = true;
+                break;
+              }
+            }
+            if (removed) {
+              continue;
+            }
             //remove expressions which are not valid in the timerange queried
             // If an expression is defined as
             // ex = a + b // from t1 to t2;

http://git-wip-us.apache.org/repos/asf/lens/blob/4fb25061/lens-cube/src/main/java/org/apache/lens/cube/parse/QueriedPhraseContext.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/QueriedPhraseContext.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/QueriedPhraseContext.java
index 11eb8f7..34a562d 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/QueriedPhraseContext.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/QueriedPhraseContext.java
@@ -116,7 +116,7 @@ class QueriedPhraseContext extends TracksQueriedColumns implements TrackQueriedC
     for (String col : queriedDimAttrs) {
       if (!cfact.getColumns().contains(col.toLowerCase())) {
         // check if it available as reference
-        if (!cubeQl.getDeNormCtx().addRefUsage(cfact, col, cubeQl.getCube().getName())) {
+        if (!cubeQl.getDeNormCtx().addRefUsage(cubeQl, cfact, col, cubeQl.getCube().getName())) {
           log.info("column {} is not available in fact table:{} ", col, cfact);
           return false;
         }

http://git-wip-us.apache.org/repos/asf/lens/blob/4fb25061/lens-cube/src/main/java/org/apache/lens/cube/parse/TimeRangeChecker.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/TimeRangeChecker.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/TimeRangeChecker.java
index 89b50f5..f18ae36 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/TimeRangeChecker.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/TimeRangeChecker.java
@@ -160,24 +160,6 @@ public class TimeRangeChecker implements ContextRewriter {
       }
     }
 
-    // Look at referenced columns through denormalization resolver
-    // and do column life validation
-    Map<String, Set<DenormalizationResolver.ReferencedQueriedColumn>> refCols =
-        cubeql.getDeNormCtx().getReferencedCols();
-    for (String col : refCols.keySet()) {
-      Iterator<DenormalizationResolver.ReferencedQueriedColumn> refColIter = refCols.get(col).iterator();
-      while (refColIter.hasNext()) {
-        DenormalizationResolver.ReferencedQueriedColumn refCol = refColIter.next();
-        for (TimeRange range : cubeql.getTimeRanges()) {
-          if (!refCol.col.isColumnAvailableInTimeRange(range)) {
-            log.debug("The refernced column: {} is not in the range queried", refCol.col.getName());
-            refColIter.remove();
-            break;
-          }
-        }
-      }
-    }
-
     // Remove join paths that have columns with invalid life span
     AutoJoinContext joinContext = cubeql.getAutoJoinCtx();
     if (joinContext == null) {

http://git-wip-us.apache.org/repos/asf/lens/blob/4fb25061/lens-cube/src/main/java/org/apache/lens/cube/parse/TrackDenormContext.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/TrackDenormContext.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/TrackDenormContext.java
new file mode 100644
index 0000000..5592f70
--- /dev/null
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/TrackDenormContext.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.cube.parse;
+
+interface TrackDenormContext {
+
+  /**
+   * Get denormalization context
+   *
+   * @return DenormalizationContext
+   */
+  DenormalizationResolver.DenormalizationContext getDeNormCtx();
+
+  /**
+   * Set denormalization context
+   *
+   * @param deNormCtx DenormalizationContext
+   */
+  void setDeNormCtx(DenormalizationResolver.DenormalizationContext deNormCtx);
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/4fb25061/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
index 41ea83d..9b29083 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
@@ -578,6 +578,8 @@ public class CubeTestSetup {
       "dim3 refer", "dim3chain", "id", null, null, 0.0));
     cubeDimensions.add(new ReferencedDimAttribute(new FieldSchema("cityname", "string", "city name"),
       "city name", "cubecity", "name", null, null, 0.0));
+    cubeDimensions.add(new ReferencedDimAttribute(new FieldSchema("statename_cube", "string", "state name"),
+      "state name", "cubestate", "name", null, null, 0.0));
     List<ChainRefCol> references = new ArrayList<>();
     references.add(new ChainRefCol("timedatechain1", "full_date"));
     references.add(new ChainRefCol("timehourchain1", "full_hour"));
@@ -592,6 +594,8 @@ public class CubeTestSetup {
       "City1", null, null, null));
     cubeDimensions.add(new BaseDimAttribute(new FieldSchema("cityid2", "int", "id to city"),
       "City2", null, null, null));
+    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("concatedcitystate", "string", "citystate"),
+      "CityState", null, null, null));
 
     Map<String, JoinChain> joinChains = new HashMap<>();
     addCubeChains(joinChains, TEST_CUBE_NAME);
@@ -653,7 +657,11 @@ public class CubeTestSetup {
     exprs.add(new ExprColumn(new FieldSchema("newexpr", "string", "expression which non existing colun"),
       "new measure expr", "myfun(newmeasure)"));
     exprs.add(new ExprColumn(new FieldSchema("cityAndState", "String", "city and state together"), "City and State",
-      "concat(cubecity.name, \":\", cubestate.name)"));
+      new ExprSpec("concat(cityname, \":\", statename_cube)", null, null),
+      new ExprSpec("substr(concatedcitystate, 10)", null, null)));
+    exprs.add(new ExprColumn(new FieldSchema("cityAndStateNew", "String", "city and state together"), "City and State",
+      new ExprSpec("concat(cityname, \":\", statename_cube)", null, TWO_MONTHS_BACK),
+      new ExprSpec("substr(concatedcitystate, 10)", null, null)));
     exprs.add(new ExprColumn(new FieldSchema("cityStateName", "String", "city state"), "City State",
       "concat('CityState:', cubecity.statename)"));
     exprs.add(new ExprColumn(new FieldSchema("isIndia", "String", "is indian city/state"), "Is Indian City/state",
@@ -1957,6 +1965,7 @@ public class CubeTestSetup {
     factColumns.add(new FieldSchema("countryid", "int", "country id"));
     factColumns.add(new FieldSchema("dim1", "string", "dim1"));
     factColumns.add(new FieldSchema("dim2", "int", "dim2"));
+    factColumns.add(new FieldSchema("concatedCityState", "string", "citystate"));
 
     Map<String, Set<UpdatePeriod>> storageAggregatePeriods = new HashMap<String, Set<UpdatePeriod>>();
     Set<UpdatePeriod> updates = new HashSet<UpdatePeriod>();

http://git-wip-us.apache.org/repos/asf/lens/blob/4fb25061/lens-cube/src/test/java/org/apache/lens/cube/parse/TestExpressionResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestExpressionResolver.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestExpressionResolver.java
index f2bb485..5d4e87f 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestExpressionResolver.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestExpressionResolver.java
@@ -215,6 +215,21 @@ public class TestExpressionResolver extends TestQueryRewrite {
           + " group by concat(cubecity.name, \":\", cubestate.name)", null, getWhereForHourly2days("C1_testfact2_raw"));
     TestCubeRewriter.compareQueries(hqlQuery, expected);
   }
+
+  @Test
+  public void testExpressionToExcludeJoin() throws Exception {
+    // expression which results in join
+    String hqlQuery =
+      rewrite("select cityAndStateNew, avgmsr from testCube" + " where " + TWO_DAYS_RANGE + " and substrexpr != 'XYZ'",
+        conf);
+
+    String expected =
+      getExpectedQuery(cubeName, "select substr(testcube.concatedcitystate, 10)"
+        + " avg(testcube.msr1 + testcube.msr2) FROM ", null, null, " and substr(testcube.dim1, 3) != 'XYZ'"
+        + " group by substr(testcube.concatedcitystate, 10)", null, getWhereForHourly2days("C1_testfact2_raw"));
+    TestCubeRewriter.compareQueries(hqlQuery, expected);
+  }
+
   @Test
   public void testExpressionInWhereWithJoinClausePassed() throws Exception {
     assertLensExceptionInRewrite("select cityAndState, avgmsr from testCube tc join citydim cd join statedim sd where "

http://git-wip-us.apache.org/repos/asf/lens/blob/4fb25061/lens-cube/src/test/java/org/apache/lens/cube/parse/TestJoinResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestJoinResolver.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestJoinResolver.java
index 6430ed1..677d641 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestJoinResolver.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestJoinResolver.java
@@ -545,7 +545,7 @@ public class TestJoinResolver extends TestQueryRewrite {
     Configuration conf = new Configuration(hconf);
     conf.set(CubeQueryConfUtil.DRIVER_SUPPORTED_STORAGES, "C3, C4");
     String failingQuery = "select testDim2.cityname, testDim2.cityStateCapital FROM testDim2 where " + TWO_DAYS_RANGE;
-    assertLensExceptionInRewrite(failingQuery, conf, LensCubeErrorCode.NO_REF_COL_AVAILABLE);
+    assertLensExceptionInRewrite(failingQuery, conf, LensCubeErrorCode.NO_DIM_HAS_COLUMN);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lens/blob/4fb25061/tools/conf/server/logback.xml
----------------------------------------------------------------------
diff --git a/tools/conf/server/logback.xml b/tools/conf/server/logback.xml
index 63ab23c..5173321 100644
--- a/tools/conf/server/logback.xml
+++ b/tools/conf/server/logback.xml
@@ -41,7 +41,7 @@
       <maxHistory>30</maxHistory>
     </rollingPolicy>
     <encoder>
-      <pattern>%d{dd MMM yyyy HH:mm:ss,SSS} [%X{logSegregationId}] [%t] %-5p %c - %m%n</pattern>
+      <pattern>%d{dd MMM yyyy HH:mm:ss,SSS} [%X{logSegregationId}] [%t] %-5p %c %L - %m%n</pattern>
     </encoder>
   </appender>
 
@@ -107,7 +107,7 @@
         <file>${lens.log.dir}/${queryLogId}.log</file>
         <append>true</append>
         <layout class="ch.qos.logback.classic.PatternLayout">
-          <pattern>%d{dd MMM yyyy HH:mm:ss,SSS} [%X{logSegregationId}] [%t] %-5p %c - %m%n</pattern>
+          <pattern>%d{dd MMM yyyy HH:mm:ss,SSS} [%X{logSegregationId}] [%t] %-5p %c %L - %m%n</pattern>
         </layout>
       </appender>
     </sift>


[03/16] lens git commit: LENS-1370 : Fix purging queries which are before retry framework

Posted by pr...@apache.org.
LENS-1370 : Fix purging queries which are before retry framework


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

Branch: refs/heads/lens-1381
Commit: 5ac688048ef2f8a233f022311dd8bbbd07aa0757
Parents: 4fb2506
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Wed Jan 18 17:49:07 2017 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Wed Jan 18 17:49:07 2017 +0530

----------------------------------------------------------------------
 .../main/java/org/apache/lens/server/query/LensServerDAO.java  | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/5ac68804/lens-server/src/main/java/org/apache/lens/server/query/LensServerDAO.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/query/LensServerDAO.java b/lens-server/src/main/java/org/apache/lens/server/query/LensServerDAO.java
index dd489e8..cc6ca7d 100644
--- a/lens-server/src/main/java/org/apache/lens/server/query/LensServerDAO.java
+++ b/lens-server/src/main/java/org/apache/lens/server/query/LensServerDAO.java
@@ -139,8 +139,10 @@ public class LensServerDAO {
             query.getDriverEndTime(), query.getDriverName(), query.getQueryName(), query.getSubmissionTime(),
             query.getDriverQuery(), serializeConf(query.getConf()),
             query.getFailedAttempts() == null ? 0 : query.getFailedAttempts().size());
-        for (int i = 0; i < query.getFailedAttempts().size(); i++) {
-          insertFailedAttempt(runner, conn, query.getHandle(), query.getFailedAttempts().get(i), i);
+        if (query.getFailedAttempts() != null) {
+          for (int i = 0; i < query.getFailedAttempts().size(); i++) {
+            insertFailedAttempt(runner, conn, query.getHandle(), query.getFailedAttempts().get(i), i);
+          }
         }
         conn.commit();
       } finally {


[13/16] lens git commit: LENS-1390 : Fix issues with updatePeriodTableDescriptor

Posted by pr...@apache.org.
LENS-1390 : Fix issues with updatePeriodTableDescriptor


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

Branch: refs/heads/lens-1381
Commit: bd344c7ff0f7f3eebe16ca9457d2fed2f29b61da
Parents: f0dadd7
Author: Lavkesh Lahngir <la...@linux.com>
Authored: Tue Feb 21 09:31:38 2017 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Tue Feb 21 09:31:38 2017 +0530

----------------------------------------------------------------------
 lens-api/src/main/resources/cube-0.1.xsd                     | 6 +++---
 lens-examples/src/test/resources/yaml/city_table.yaml        | 3 ++-
 lens-examples/src/test/resources/yaml/customer_table.yaml    | 3 ++-
 lens-examples/src/test/resources/yaml/dim_table.yaml         | 3 ++-
 lens-examples/src/test/resources/yaml/dim_table2.yaml        | 3 ++-
 lens-examples/src/test/resources/yaml/dim_table4.yaml        | 3 ++-
 lens-examples/src/test/resources/yaml/fact1.yaml             | 3 ++-
 lens-examples/src/test/resources/yaml/fact2.yaml             | 3 ++-
 lens-examples/src/test/resources/yaml/product_table.yaml     | 3 ++-
 lens-examples/src/test/resources/yaml/rawfact.yaml           | 3 ++-
 .../src/test/resources/yaml/sales-aggr-continuous-fact.yaml  | 3 ++-
 lens-examples/src/test/resources/yaml/sales-aggr-fact1.yaml  | 6 ++++--
 lens-examples/src/test/resources/yaml/sales-aggr-fact2.yaml  | 6 ++++--
 lens-examples/src/test/resources/yaml/sales-raw-fact.yaml    | 3 ++-
 .../java/org/apache/lens/server/metastore/JAXBUtils.java     | 8 +++++---
 15 files changed, 38 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-api/src/main/resources/cube-0.1.xsd
----------------------------------------------------------------------
diff --git a/lens-api/src/main/resources/cube-0.1.xsd b/lens-api/src/main/resources/cube-0.1.xsd
index 431d68b..060eb43 100644
--- a/lens-api/src/main/resources/cube-0.1.xsd
+++ b/lens-api/src/main/resources/cube-0.1.xsd
@@ -1024,11 +1024,11 @@
         of update_period_table_descriptor instead of a list of enums.
       </xs:documentation>
     </xs:annotation>
-    <xs:sequence>
-      <xs:element name="update_periods" type="x_update_periods" maxOccurs="1" minOccurs="1"/>
+    <xs:all>
+      <xs:element name="update_periods" type="x_update_periods" maxOccurs="1" minOccurs="0"/>
       <xs:element name="storage_name" type="xs:string"/>
       <xs:element type="x_storage_table_desc" name="table_desc" maxOccurs="1" minOccurs="0"/>
-    </xs:sequence>
+    </xs:all>
   </xs:complexType>
 
   <xs:complexType name="x_storage_tables">

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/city_table.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/city_table.yaml b/lens-examples/src/test/resources/yaml/city_table.yaml
index 4f3b986..92ce3ec 100644
--- a/lens-examples/src/test/resources/yaml/city_table.yaml
+++ b/lens-examples/src/test/resources/yaml/city_table.yaml
@@ -22,7 +22,8 @@ properties:
   city.prop: d1
 storageTables:
   local:
-    updatePeriods: HOURLY
+    updatePeriods:
+      updatePeriod: HOURLY
     tableDesc:
       partCols:
         dt: type: STRING, comment: Time column

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/customer_table.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/customer_table.yaml b/lens-examples/src/test/resources/yaml/customer_table.yaml
index 2de5cd9..4209bea 100644
--- a/lens-examples/src/test/resources/yaml/customer_table.yaml
+++ b/lens-examples/src/test/resources/yaml/customer_table.yaml
@@ -24,7 +24,8 @@ properties:
   dim4.prop: d1
 storageTables:
   local:
-    updatePeriods: HOURLY
+    updatePeriods:
+      updatePeriod: HOURLY
     tableDesc:
       partCols:
         dt: type: STRING, comment: Time column

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/dim_table.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/dim_table.yaml b/lens-examples/src/test/resources/yaml/dim_table.yaml
index 8191456..37b50da 100644
--- a/lens-examples/src/test/resources/yaml/dim_table.yaml
+++ b/lens-examples/src/test/resources/yaml/dim_table.yaml
@@ -22,7 +22,8 @@ properties:
   dim1.prop: d1
 storageTables:
   local:
-    updatePeriods: HOURLY
+    updatePeriods:
+      updatePeriod: HOURLY
     tableDesc:
       partCols:
         dt: type: STRING, comment: Time column

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/dim_table2.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/dim_table2.yaml b/lens-examples/src/test/resources/yaml/dim_table2.yaml
index 0df4682..619625c 100644
--- a/lens-examples/src/test/resources/yaml/dim_table2.yaml
+++ b/lens-examples/src/test/resources/yaml/dim_table2.yaml
@@ -21,7 +21,8 @@ properties:
   dim2.prop: d2
 storageTables:
   local:
-    updatePeriods: HOURLY
+    updatePeriods:
+      updatePeriod: HOURLY
     tableDesc:
       partCols:
         dt: type: STRING, comment: Time column

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/dim_table4.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/dim_table4.yaml b/lens-examples/src/test/resources/yaml/dim_table4.yaml
index f26c2cd..03b2809 100644
--- a/lens-examples/src/test/resources/yaml/dim_table4.yaml
+++ b/lens-examples/src/test/resources/yaml/dim_table4.yaml
@@ -22,7 +22,8 @@ properties:
   dim4.prop: d1
 storageTables:
   local:
-    updatePeriods: HOURLY
+    updatePeriods:
+      updatePeriod: HOURLY
     tableDesc:
       partCols:
         dt: type: STRING, comment: Time column

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/fact1.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/fact1.yaml b/lens-examples/src/test/resources/yaml/fact1.yaml
index c5c6d57..9c1c527 100644
--- a/lens-examples/src/test/resources/yaml/fact1.yaml
+++ b/lens-examples/src/test/resources/yaml/fact1.yaml
@@ -28,7 +28,8 @@ properties:
   cube.fact.is.aggregated: true
 storageTables:
   local:
-    updatePeriods: HOURLY, DAILY
+    updatePeriods:
+      updatePeriod: HOURLY, DAILY
     tableDesc:
       partCols:
         dt: type: STRING, comment: Time column

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/fact2.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/fact2.yaml b/lens-examples/src/test/resources/yaml/fact2.yaml
index 97112c8..6fadd11 100644
--- a/lens-examples/src/test/resources/yaml/fact2.yaml
+++ b/lens-examples/src/test/resources/yaml/fact2.yaml
@@ -23,7 +23,8 @@ properties:
   cube.fact.is.aggregated: true
 storageTables:
   local:
-    updatePeriods: HOURLY, DAILY
+    updatePeriods:
+      updatePeriod: HOURLY, DAILY
     tableDesc:
       partCols:
         dt: type: STRING, comment: Time column

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/product_table.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/product_table.yaml b/lens-examples/src/test/resources/yaml/product_table.yaml
index 54d37d8..637db62 100644
--- a/lens-examples/src/test/resources/yaml/product_table.yaml
+++ b/lens-examples/src/test/resources/yaml/product_table.yaml
@@ -25,7 +25,8 @@ properties:
   dimtable.product_table.part.cols: category
 storageTables:
   local:
-    updatePeriods: HOURLY
+    updatePeriods:
+      updatePeriod: HOURLY
     tableDesc:
       partCols:
         dt: type: STRING, comment: Time column

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/rawfact.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/rawfact.yaml b/lens-examples/src/test/resources/yaml/rawfact.yaml
index f750192..f7b2d55 100644
--- a/lens-examples/src/test/resources/yaml/rawfact.yaml
+++ b/lens-examples/src/test/resources/yaml/rawfact.yaml
@@ -25,7 +25,8 @@ properties:
   cube.fact.is.aggregated: false
 storageTables:
   local:
-    updatePeriods: HOURLY, DAILY
+    updatePeriods:
+      updatePeriod: HOURLY, DAILY
     tableDesc:
       partCols:
         dt: type: STRING, comment: Time column

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/sales-aggr-continuous-fact.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/sales-aggr-continuous-fact.yaml b/lens-examples/src/test/resources/yaml/sales-aggr-continuous-fact.yaml
index 0e98281..f95f91b 100644
--- a/lens-examples/src/test/resources/yaml/sales-aggr-continuous-fact.yaml
+++ b/lens-examples/src/test/resources/yaml/sales-aggr-continuous-fact.yaml
@@ -32,7 +32,8 @@ properties:
   cube.fact.relative.start.time: now.second - 2 days
 storageTables:
   mydb:
-    updatePeriods: CONTINUOUS
+    updatePeriods:
+      updatePeriod: CONTINUOUS
     tableDesc:
       tableParameters:
         lens.metastore.native.db.name: default

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/sales-aggr-fact1.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/sales-aggr-fact1.yaml b/lens-examples/src/test/resources/yaml/sales-aggr-fact1.yaml
index 0be8cf2..02f65ff 100644
--- a/lens-examples/src/test/resources/yaml/sales-aggr-fact1.yaml
+++ b/lens-examples/src/test/resources/yaml/sales-aggr-fact1.yaml
@@ -31,7 +31,8 @@ properties:
   cube.fact.is.aggregated: true
 storageTables:
   local:
-    updatePeriods: HOURLY, DAILY
+    updatePeriods:
+      updatePeriod: HOURLY, DAILY
     tableDesc:
       partCols:
         pt: type: STRING, comment: Process time partition
@@ -42,7 +43,8 @@ storageTables:
       tableLocation: /tmp/examples/aggrfact1
       fieldDelimiter: ,
   mydb:
-    updatePeriods: DAILY
+    updatePeriods:
+      updatePeriod: DAILY
     tableDesc:
       partCols:
         pt: type: STRING, comment: Process time partition

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/sales-aggr-fact2.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/sales-aggr-fact2.yaml b/lens-examples/src/test/resources/yaml/sales-aggr-fact2.yaml
index d1ba7fb..ff47d36 100644
--- a/lens-examples/src/test/resources/yaml/sales-aggr-fact2.yaml
+++ b/lens-examples/src/test/resources/yaml/sales-aggr-fact2.yaml
@@ -29,7 +29,8 @@ properties:
   cube.timedim.relation.order_time: delivery_time+[-2 hours,-1hour]
 storageTables:
   local:
-    updatePeriods: HOURLY, DAILY
+    updatePeriods:
+      updatePeriod: HOURLY, DAILY
     tableDesc:
       partCols:
         dt: type: STRING, comment: Delivery time partition
@@ -38,7 +39,8 @@ storageTables:
       tableLocation: /tmp/examples/aggrfact2
       fieldDelimiter: ,
   mydb:
-    updatePeriods: DAILY
+    updatePeriods:
+      updatePeriod: DAILY
     tableDesc:
       partCols:
         pt: type: STRING, comment: Process time partition

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-examples/src/test/resources/yaml/sales-raw-fact.yaml
----------------------------------------------------------------------
diff --git a/lens-examples/src/test/resources/yaml/sales-raw-fact.yaml b/lens-examples/src/test/resources/yaml/sales-raw-fact.yaml
index eaf7f15..15becfc 100644
--- a/lens-examples/src/test/resources/yaml/sales-raw-fact.yaml
+++ b/lens-examples/src/test/resources/yaml/sales-raw-fact.yaml
@@ -35,7 +35,8 @@ properties:
   cube.fact.is.aggregated: false
 storageTables:
   local:
-    updatePeriods: HOURLY
+    updatePeriods:
+      updatePeriod: HOURLY
     tableDesc:
       partCols:
         pt: type: STRING, comment: Process time partition

http://git-wip-us.apache.org/repos/asf/lens/blob/bd344c7f/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java b/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
index 0bc8e77..7d54c7b 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
@@ -857,7 +857,8 @@ public final class JAXBUtils {
     Map<String, StorageTableDesc> storageTablePrefixToDescMap = new HashMap<>();
     if (storageTables != null && !storageTables.getStorageTable().isEmpty()) {
       for (XStorageTableElement sTbl : storageTables.getStorageTable()) {
-        if (!sTbl.getUpdatePeriods().getUpdatePeriodTableDescriptor().isEmpty()) {
+        if (sTbl.getUpdatePeriods() != null && sTbl.getUpdatePeriods().getUpdatePeriodTableDescriptor() != null && !sTbl
+          .getUpdatePeriods().getUpdatePeriodTableDescriptor().isEmpty()) {
           for (XUpdatePeriodTableDescriptor updatePeriodTable : sTbl.getUpdatePeriods()
             .getUpdatePeriodTableDescriptor()) {
             // Get table name with update period as the prefix.
@@ -877,7 +878,8 @@ public final class JAXBUtils {
     if (storageTables != null && !storageTables.getStorageTable().isEmpty()) {
       for (XStorageTableElement sTbl : storageTables.getStorageTable()) {
         Map<UpdatePeriod, String> storageNameMap = new HashMap<>();
-        if (!sTbl.getUpdatePeriods().getUpdatePeriodTableDescriptor().isEmpty()) {
+        if (sTbl.getUpdatePeriods() != null && sTbl.getUpdatePeriods().getUpdatePeriodTableDescriptor() != null && !sTbl
+          .getUpdatePeriods().getUpdatePeriodTableDescriptor().isEmpty()) {
           for (XUpdatePeriodTableDescriptor updatePeriodTable : sTbl.getUpdatePeriods()
             .getUpdatePeriodTableDescriptor()) {
             // Get table name with update period as the prefix.
@@ -885,7 +887,7 @@ public final class JAXBUtils {
               updatePeriodTable.getUpdatePeriod() + "_" + sTbl.getStorageName());
           }
         } else {
-          for (XUpdatePeriod updatePeriod :sTbl.getUpdatePeriods().getUpdatePeriod()) {
+          for (XUpdatePeriod updatePeriod : sTbl.getUpdatePeriods().getUpdatePeriod()) {
             storageNameMap.put(UpdatePeriod.valueOf(updatePeriod.value()), sTbl.getStorageName());
           }
         }


[16/16] lens git commit: LENS-1389: Back Merge with master

Posted by pr...@apache.org.
LENS-1389: Back Merge with master


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

Branch: refs/heads/lens-1381
Commit: 8868b063838050dd8746cefa491f143533e145c9
Parents: d45c538 d1b43d6
Author: Rajat Khandelwal <ra...@gmail.com>
Authored: Fri Mar 31 15:16:10 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Fri Mar 31 15:16:10 2017 +0530

----------------------------------------------------------------------
 contrib/clients/python/lens/client/main.py      |  7 +++-
 contrib/clients/python/lens/client/query.py     | 43 +++++++++++++++-----
 contrib/clients/python/setup.py                 |  2 +
 contrib/clients/python/test/test_lensclient.py  | 13 ++++--
 .../lens/cube/metadata/CubeMetastoreClient.java | 22 +++++-----
 .../cube/parse/DenormalizationResolver.java     |  1 +
 .../lens/cube/parse/ExpressionResolver.java     |  2 +-
 .../apache/lens/cube/parse/CubeTestSetup.java   |  2 -
 .../metastore/CubeMetastoreServiceImpl.java     |  2 -
 tools/conf/server/lens-site.xml                 |  6 +++
 10 files changed, 69 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/8868b063/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
----------------------------------------------------------------------
diff --cc lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
index 7608a43,087c203..b445447
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
@@@ -307,74 -290,18 +307,74 @@@ public class CubeMetastoreClient 
      }
    }
  
+   public void createCubeFactTable(String cubeName, String factName, List<FieldSchema> columns,
+     Map<String, Set<UpdatePeriod>> storageAggregatePeriods, double weight, Map<String, String> properties,
+     Map<String, StorageTableDesc> storageTableDescs, Map<String, Map<UpdatePeriod, String>> storageUpdatePeriodMap)
+     throws LensException {
+     CubeFactTable factTable = new CubeFactTable(cubeName, factName, columns, storageAggregatePeriods, weight,
+       properties, storageUpdatePeriodMap);
+     createCubeTable(factTable, storageTableDescs);
+     // do a get to update cache
+     getCubeFact(factName);
+ 
+   }
  
 +  public <T extends Equals & HashCode & ToString> void createEntity(T entity) throws LensException {
 +    if (entity instanceof XStorage) {
 +      createStorage((XStorage) entity);
 +    } else if  (entity instanceof XCube) {
 +      createCube((XCube)entity);
 +    } else if (entity instanceof XDimension) {
 +      createDimension((XDimension) entity);
 +    } else if (entity instanceof XFactTable) {
 +      createCubeFactTable((XFactTable) entity);
 +    } else if (entity instanceof XDimensionTable) {
 +      createCubeDimensionTable((XDimensionTable) entity);
 +    } else if (entity instanceof XSegmentation) {
 +      createSegmentation((XSegmentation) entity);
 +    } else {
 +      throw new LensException("Unable to create entity " + entity + " as it's unrecognizable: "+ entity.getClass());
 +    }
 +  }
 +
 +  public <T extends Equals & HashCode & ToString> void updateEntity(String name, T entity)
 +    throws LensException, HiveException {
 +    if (entity instanceof XStorage) {
 +      alterStorage((XStorage) entity);
 +    } else if  (entity instanceof XCube) {
 +      alterCube((XCube)entity);
 +    } else if (entity instanceof XDimension) {
 +      alterDimension((XDimension) entity);
 +    } else if (entity instanceof XFactTable) {
 +      alterCubeFactTable((XFactTable) entity);
 +    } else if (entity instanceof XDimensionTable) {
 +      alterCubeDimensionTable((XDimensionTable) entity);
 +    } else if (entity instanceof XSegmentation) {
 +      alterSegmentation((XSegmentation) entity);
 +    } else {
 +      throw new LensException("Unable to alter entity " + entity + " as it's unrecognizable: " + entity.getClass());
 +    }
 +  }
 +
 +
 +  public static Map<String, String> addFactColStartTimePropertyToFactProperties(XFactTable fact) {
 +    Map<String, String> props = new HashMap<String, String>();
 +    props.putAll(JAXBUtils.mapFromXProperties(fact.getProperties()));
 +    props.putAll(JAXBUtils.columnStartAndEndTimeFromXColumns(fact.getColumns()));
 +    return props;
 +  }
 +  public void createCubeFactTable(XFactTable fact) throws LensException {
 +    createCubeFactTable(fact.getCubeName(),
 +      fact.getName(),
 +      JAXBUtils.fieldSchemaListFromColumns(fact.getColumns()),
 +      JAXBUtils.getFactUpdatePeriodsFromStorageTables(fact.getStorageTables()),
 +      fact.getWeight(),
 +      addFactColStartTimePropertyToFactProperties(fact),
 +      JAXBUtils.tableDescPrefixMapFromXStorageTables(fact.getStorageTables()),
 +      JAXBUtils.storageTablePrefixMapOfStorage(fact.getStorageTables()));
 +  }
-   public void createCubeFactTable(String cubeName, String factName, List<FieldSchema> columns,
-     Map<String, Set<UpdatePeriod>> storageAggregatePeriods, double weight, Map<String, String> properties,
-     Map<String, StorageTableDesc> storageTableDescs, Map<String, Map<UpdatePeriod, String>> storageUpdatePeriodMap)
-     throws LensException {
-     CubeFactTable factTable = new CubeFactTable(cubeName, factName, columns, storageAggregatePeriods, weight,
-       properties, storageUpdatePeriodMap);
-     createCubeTable(factTable, storageTableDescs);
-     // do a get to update cache
-     getCubeFact(factName);
- 
-   }
 +
 +
    /**
     * In-memory storage of {@link PartitionTimeline} objects for each valid
     * storagetable-updateperiod-partitioncolumn tuple. also simultaneously stored in metastore table of the

http://git-wip-us.apache.org/repos/asf/lens/blob/8868b063/lens-cube/src/main/java/org/apache/lens/cube/parse/DenormalizationResolver.java
----------------------------------------------------------------------
diff --cc lens-cube/src/main/java/org/apache/lens/cube/parse/DenormalizationResolver.java
index 76e5f23,cb26878..e5cf916
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/DenormalizationResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/DenormalizationResolver.java
@@@ -138,9 -161,10 +138,10 @@@ public class DenormalizationResolver im
        return null;
      }
  
 -    public Set<Dimension> rewriteDenormctx(CubeQueryContext cubeql, CandidateFact cfact, Map<Dimension,
 -      CandidateDim> dimsToQuery, boolean replaceFact) throws LensException {
 +    Set<Dimension> rewriteDenormctx(CubeQueryContext cubeql,
 +      StorageCandidate sc, Map<Dimension, CandidateDim> dimsToQuery, boolean replaceFact) throws LensException {
        Set<Dimension> refTbls = new HashSet<>();
 -      log.info("Doing denorm changes for fact :{}", cfact);
++      log.info("Doing denorm changes for fact :{}", sc);
  
        if (!tableToRefCols.isEmpty()) {
          // pick referenced columns for fact

http://git-wip-us.apache.org/repos/asf/lens/blob/8868b063/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lens/blob/8868b063/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
----------------------------------------------------------------------
diff --cc lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
index 860db28,9b29083..62d7386
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
@@@ -525,16 -511,1143 +525,14 @@@ public class CubeTestSetup 
      return expected.toString();
    }
  
-   private Set<ExprColumn> exprs;
 -  Set<ExprColumn> exprs;
 -
 -  private void createCube(CubeMetastoreClient client) throws HiveException, ParseException, LensException {
 -    cubeMeasures = new HashSet<CubeMeasure>();
 -    Map<String, String> tags = new HashMap<>();
 -    tags.put(MetastoreConstants.MEASURE_DATACOMPLETENESS_TAG, "tag1");
 -    cubeMeasures.add(new ColumnMeasure(new FieldSchema("msr1", "int", "first measure"), null, null, null, null, null,
 -            null, null, null, null, tags));
 -    cubeMeasures.add(new ColumnMeasure(new FieldSchema("msr2", "float", "second measure"), "Measure2", null, "SUM",
 -      "RS"));
 -    cubeMeasures.add(new ColumnMeasure(new FieldSchema("msr21", "float", "second measure"), "Measure22", null, "SUM",
 -      "RS"));
 -    cubeMeasures.add(new ColumnMeasure(new FieldSchema("msr22", "float", "second measure"), "Measure22", null, "SUM",
 -      "RS"));
 -    cubeMeasures.add(new ColumnMeasure(new FieldSchema("msr3", "double", "third measure"), "Measure3", null, "MAX",
 -      null));
 -    cubeMeasures.add(new ColumnMeasure(new FieldSchema("msr4", "bigint", "fourth measure"), "Measure4", null, "COUNT",
 -      null));
 -    cubeMeasures.add(new ColumnMeasure(new FieldSchema("msr9", "bigint", "ninth measure"), null, null, null, null,
 -            null, null, null, null, null, tags));
 -    cubeMeasures.add(new ColumnMeasure(new FieldSchema("noAggrMsr", "bigint", "measure without a default aggregate"),
 -      "No aggregateMsr", null, null, null));
 -    cubeMeasures.add(new ColumnMeasure(new FieldSchema("newmeasure", "bigint", "measure available  from now"),
 -      "New measure", null, null, null, NOW, null, 100.0));
 -    cubeMeasures.add(new ColumnMeasure(new FieldSchema("msr15", "int", "fifteenth measure"), "Measure15", null, "SUM",
 -      "RS"));
 -
 -    cubeDimensions = new HashSet<CubeDimAttribute>();
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("d_time", "timestamp", "d time")));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("processing_time", "timestamp", "processing time")));
 -    List<CubeDimAttribute> locationHierarchy = new ArrayList<CubeDimAttribute>();
 -    locationHierarchy.add(new BaseDimAttribute(new FieldSchema("zipcode", "int", "zip")));
 -    locationHierarchy.add(new BaseDimAttribute(new FieldSchema("cityid", "int", "city")));
 -    locationHierarchy.add(new BaseDimAttribute(new FieldSchema("stateid", "int", "state")));
 -    locationHierarchy.add(new BaseDimAttribute(new FieldSchema("countryid", "int", "country")));
 -    List<String> regions = Arrays.asList("APAC", "EMEA", "USA");
 -    locationHierarchy.add(new BaseDimAttribute(new FieldSchema("regionname", "string", "region"), "regionname", null,
 -      null, null, null, regions));
 -
 -    cubeDimensions.add(new HierarchicalDimAttribute("location", "Location hierarchy", locationHierarchy));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("dim1", "string", "basedim")));
 -    // Added for ambiguity test
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("ambigdim1", "string", "used in testColumnAmbiguity")));
 -    cubeDimensions.add(new ReferencedDimAttribute(new FieldSchema("dim2", "int", "ref dim"), "dim2 refer",
 -      "dim2chain", "id", null, null, 0.0));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("cdim2", "int", "ref dim"), "Dim2 refer", NOW, null, null));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("urdimid", "int", "ref dim"), "urdim refer",
 -      null, null, 10.0));
 -    cubeDimensions.add(new ReferencedDimAttribute(new FieldSchema("unreachableName", "string", ""), "urdim name",
 -      "unreachableDim_chain", "name", null, null, 10.0));
 -    // denormalized reference
 -    cubeDimensions.add(new ReferencedDimAttribute(new FieldSchema("dim2big1", "bigint", "ref dim"), "dim2 refer",
 -      "dim2chain", "bigid1", null, null, 0.0));
 -    cubeDimensions.add(new ReferencedDimAttribute(new FieldSchema("dim2big2", "bigint", "ref dim"), "dim2 refer",
 -      "dim2chain", "bigid2", null, null, 0.0));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("dim2bignew", "bigint", "ref dim"), "Dim2 refer",
 -      NOW, null, null));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("test_time_dim_hour_id", "int", "ref dim"),
 -      "Timedim reference", null, null, null));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("test_time_dim_day_id", "int", "ref dim"),
 -      "Timedim reference", null, null, null));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("test_time_dim_hour_id2", "int", "ref dim")));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("test_time_dim_day_id2", "int", "ref dim")));
 -    cubeDimensions.add(new ReferencedDimAttribute(new FieldSchema("testDim3id", "string", "direct id to testdim3"),
 -      "dim3 refer", "dim3chain", "id", null, null, 0.0));
 -    cubeDimensions.add(new ReferencedDimAttribute(new FieldSchema("cityname", "string", "city name"),
 -      "city name", "cubecity", "name", null, null, 0.0));
 -    cubeDimensions.add(new ReferencedDimAttribute(new FieldSchema("statename_cube", "string", "state name"),
 -      "state name", "cubestate", "name", null, null, 0.0));
 -    List<ChainRefCol> references = new ArrayList<>();
 -    references.add(new ChainRefCol("timedatechain1", "full_date"));
 -    references.add(new ChainRefCol("timehourchain1", "full_hour"));
 -    cubeDimensions.add(new ReferencedDimAttribute(new FieldSchema("test_time_dim", "date", "ref dim"),
 -      "Timedim full date", references, null, null, null, null));
 -    List<ChainRefCol> chainRefs = new ArrayList<>();
 -    chainRefs.add(new ChainRefCol("timehourchain2", "full_hour"));
 -    chainRefs.add(new ChainRefCol("timedatechain2", "full_date"));
 -    cubeDimensions.add(new ReferencedDimAttribute(new FieldSchema("test_time_dim2", "date", "chained dim"),
 -      "Timedim full date", chainRefs, null, null, null, null));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("cityid1", "int", "id to city"),
 -      "City1", null, null, null));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("cityid2", "int", "id to city"),
 -      "City2", null, null, null));
 -    cubeDimensions.add(new BaseDimAttribute(new FieldSchema("concatedcitystate", "string", "citystate"),
 -      "CityState", null, null, null));
 -
 -    Map<String, JoinChain> joinChains = new HashMap<>();
 -    addCubeChains(joinChains, TEST_CUBE_NAME);
 -
 -    exprs = new HashSet<ExprColumn>();
 -    exprs.add(new ExprColumn(new FieldSchema("avgmsr", "double", "avg measure"), "Avg Msr", "avg(msr1 + msr2)"));
 -    exprs.add(new ExprColumn(new FieldSchema("singlecolmsr2expr", "double", "measure2"), "Msr2", "msr2)"));
 -    exprs.add(new ExprColumn(new FieldSchema("singlecolmsr2qualifiedexpr", "double", "testcube.measure2"),
 -      "Msr2", "testcube.msr2"));
 -    exprs.add(new ExprColumn(new FieldSchema("singlecoldim1expr", "string", "dim1"), "dim1", "dim1)"));
 -    exprs.add(new ExprColumn(new FieldSchema("singlecoldim1qualifiedexpr", "string", "testcube.dim1"),
 -      "dim1", "testcube.dim1"));
 -    exprs.add(new ExprColumn(new FieldSchema("singlecolchainid", "string", "dim3chain.id"),
 -      "dim3chainid", "dim3chain.id)"));
 -    exprs.add(new ExprColumn(new FieldSchema("singlecolchainrefexpr", "string", "testcube.testDim3id"),
 -      "dim3chainid", "testcube.testDim3id"));
 -    exprs.add(new ExprColumn(new FieldSchema("singlecolchainfield", "string", "cubecity.name"),
 -      "cubecityname", "cubecity.name"));
 -    exprs.add(new ExprColumn(new FieldSchema("summsrs", "double", "sum measures"), "Sum Msrs",
 -      "(1000 + sum(msr1) + sum(msr2))/100"));
 -    exprs.add(new ExprColumn(new FieldSchema("msr5", "double", "materialized in some facts"), "Fifth Msr",
 -      "msr2 + msr3"));
 -    exprs.add(new ExprColumn(new FieldSchema("msr8", "double", "measure expression"), "Sixth Msr",
 -      "msr2 + msr3"));
 -    exprs.add(new ExprColumn(new FieldSchema("msr7", "double", "measure expression"), "Seventh Msr",
 -      "case when sum(msr2) = 0 then 0 else sum(case when cityid='x' then msr21 else msr22 end)/sum(msr2) end"));
 -    exprs.add(new ExprColumn(new FieldSchema("equalsums", "double", "sums are equals"), "equalsums",
 -      new ExprSpec("msr3 + msr4", null, null), new ExprSpec("(msr3 + msr2)/100", null, null)));
 -    exprs.add(new ExprColumn(new FieldSchema("roundedmsr1", "double", "rounded measure1"), "Rounded msr1",
 -      "round(msr1/1000)"));
 -    exprs.add(new ExprColumn(new FieldSchema("roundedmsr2", "double", "rounded measure2"), "Rounded msr2",
 -      "round(msr2/1000)"));
 -    exprs.add(new ExprColumn(new FieldSchema("flooredmsr12", "double", "floored measure12"), "Floored msr12",
 -            "floor(msr12)"));
 -    exprs.add(new ExprColumn(new FieldSchema("nestedexpr", "double", "nested expr"), "Nested expr",
 -      new ExprSpec("avg(roundedmsr2)", null, null), new ExprSpec("avg(equalsums)", null, null),
 -      new ExprSpec("case when substrexpr = 'xyz' then avg(msr5) when substrexpr = 'abc' then avg(msr4)/100 end",
 -        null, null)));
 -    exprs.add(new ExprColumn(new FieldSchema("msr2expr", "double", "nested expr"), "Nested expr",
 -      new ExprSpec("case when cityStateName = 'xyz' then msr2 else 0 end", null, null)));
 -    exprs.add(new ExprColumn(new FieldSchema("nestedExprWithTimes", "double", "nested expr"), "Nested expr",
 -      new ExprSpec("avg(roundedmsr2)", null, null), new ExprSpec("avg(equalsums)", null, null),
 -      new ExprSpec("case when substrexpr = 'xyz' then avg(msr5) when substrexpr = 'abc' then avg(msr4)/100 end",
 -        NOW, null), new ExprSpec("avg(newmeasure)", null, null)));
 -    exprs.add(new ExprColumn(new FieldSchema("msr6", "bigint", "sixth measure"), "Measure6",
 -      "sum(msr2) + max(msr3)/ count(msr4)"));
 -    exprs.add(new ExprColumn(new FieldSchema("booleancut", "boolean", "a boolean expression"), "Boolean cut",
 -      "(dim1 != 'x' AND dim2 != 10)"));
 -    exprs.add(new ExprColumn(new FieldSchema("substrexpr", "string", "a sub-string expression"), "Substr expr",
 -      new ExprSpec("substr(dim1, 3))", null, null), new ExprSpec("substr(ascii(dim2chain.name), 3)", null, null)));
 -    exprs.add(new ExprColumn(new FieldSchema("substrexprdim2", "string", "a sub-string expression"), "Substr expr",
 -      new ExprSpec("substr(dim2, 3))", null, null), new ExprSpec("substr(ascii(dim2chain.name), 3)", null, null)));
 -    exprs.add(new ExprColumn(new FieldSchema("indiasubstr", "boolean", "nested sub string expression"), "Nested expr",
 -      "substrexpr = 'INDIA'"));
 -    exprs.add(new ExprColumn(new FieldSchema("refexpr", "string", "expression which facts and dimensions"),
 -      "Expr with cube and dim fields", "concat(dim1, \":\", citydim.name)"));
 -    exprs.add(new ExprColumn(new FieldSchema("nocolexpr", "string", "expression which non existing colun"),
 -      "No col expr", "myfun(nonexist)"));
 -    exprs.add(new ExprColumn(new FieldSchema("newexpr", "string", "expression which non existing colun"),
 -      "new measure expr", "myfun(newmeasure)"));
 -    exprs.add(new ExprColumn(new FieldSchema("cityAndState", "String", "city and state together"), "City and State",
 -      new ExprSpec("concat(cityname, \":\", statename_cube)", null, null),
 -      new ExprSpec("substr(concatedcitystate, 10)", null, null)));
 -    exprs.add(new ExprColumn(new FieldSchema("cityAndStateNew", "String", "city and state together"), "City and State",
 -      new ExprSpec("concat(cityname, \":\", statename_cube)", null, TWO_MONTHS_BACK),
 -      new ExprSpec("substr(concatedcitystate, 10)", null, null)));
 -    exprs.add(new ExprColumn(new FieldSchema("cityStateName", "String", "city state"), "City State",
 -      "concat('CityState:', cubecity.statename)"));
 -    exprs.add(new ExprColumn(new FieldSchema("isIndia", "String", "is indian city/state"), "Is Indian City/state",
 -      "cubecity.name == 'DELHI' OR cubestate.name == 'KARNATAKA' OR cubestate.name == 'MAHARASHTRA'"));
 -    exprs.add(new ExprColumn(new FieldSchema("cubeStateName", "String", "statename from cubestate"), "CubeState Name",
 -      "substr(cubestate.name, 5)"));
 -    exprs.add(new ExprColumn(new FieldSchema("substrdim2big1", "String", "substr of dim2big1"), "dim2big1 substr",
 -      "substr(dim2big1, 5)"));
 -    exprs.add(new ExprColumn(new FieldSchema("asciicity", "String", "ascii cityname"), "ascii cityname substr",
 -      "ascii(cityname)"));
 -    exprs.add(new ExprColumn(new FieldSchema("countofdistinctcityid", "int", "Count of Distinct CityId"),
 -        "Count of Distinct CityId Expr", "count(distinct(cityid))"));
 -    exprs.add(new ExprColumn(new FieldSchema("notnullcityid", "int", "Not null cityid"),
 -        "Not null cityid Expr", "case when cityid is null then 0 else cityid end"));
 -
 -    Map<String, String> cubeProperties = new HashMap<String, String>();
 -    cubeProperties.put(MetastoreUtil.getCubeTimedDimensionListKey(TEST_CUBE_NAME),
 -      "d_time,pt,it,et,test_time_dim,test_time_dim2");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "test_time_dim", "ttd");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "test_time_dim2", "ttd2");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "d_time", "dt");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "it", "it");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "et", "et");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "pt", "pt");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_RELATION + "d_time", "test_time_dim+[-10 days,10 days]");
 -
 -    client.createCube(TEST_CUBE_NAME, cubeMeasures, cubeDimensions, exprs, Sets.newHashSet(joinChains.values()),
 -      cubeProperties);
 -
 -    Set<String> measures = new HashSet<String>();
 -    measures.add("msr1");
 -    measures.add("msr2");
 -    measures.add("msr3");
 -    measures.add("msr9");
 -    Set<String> dimensions = new HashSet<String>();
 -    dimensions.add("dim1");
 -    dimensions.add("dim2");
 -    dimensions.add("dim2big1");
 -    dimensions.add("dim2big2");
 -    dimensions.add("dim2bignew");
 -    // Try creating derived cube with non existant dim/measures
 -    try{
 -      client.createDerivedCube(TEST_CUBE_NAME, DERIVED_CUBE_NAME,
 -        Sets.newHashSet("random_measure"), Sets.newHashSet("random_dim_attribute"),
 -        new HashMap<String, String>(), 5L);
 -    } catch(LensException e) {
 -      assertTrue(e.getMessage().contains("random_measure"));
 -      assertTrue(e.getMessage().contains("random_dim_attribute"));
 -      assertTrue(e.getMessage().contains("not present"));
 -    }
 -    client.createDerivedCube(TEST_CUBE_NAME, DERIVED_CUBE_NAME,
 -      measures, dimensions, new HashMap<String, String>(), 5L);
 -  }
 -
 -  private void addCubeChains(Map<String, JoinChain> joinChains, final String cubeName) {
 -    joinChains.put("timehourchain1", new JoinChain("timehourchain1", "time chain", "time dim thru hour dim") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "test_time_dim_hour_id"));
 -            add(new TableReference("hourdim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("timedatechain1", new JoinChain("timedatechain1", "time chain", "time dim thru date dim") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "test_time_dim_day_id"));
 -            add(new TableReference("daydim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("timehourchain2", new JoinChain("timehourchain2", "time chain", "time dim thru hour dim") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "test_time_dim_hour_id2"));
 -            add(new TableReference("hourdim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("timedatechain2", new JoinChain("timedatechain2", "time chain", "time dim thru date dim") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "test_time_dim_day_id2"));
 -            add(new TableReference("daydim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("cubeCity", new JoinChain("cubeCity", "cube-city", "city thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "cityid"));
 -            add(new TableReference("citydim", "id"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2"));
 -            add(new TableReference("testdim2", "id"));
 -            add(new TableReference("testdim2", "cityid"));
 -            add(new TableReference("citydim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("cubeCity1", new JoinChain("cubeCity1", "cube-city", "city thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "cityid1"));
 -            add(new TableReference("citydim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("cubeCity2", new JoinChain("cubeCity2", "cube-city", "city thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "cityid2"));
 -            add(new TableReference("citydim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("cubeState",  new JoinChain("cubeState", "cube-state", "state thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "stateid"));
 -            add(new TableReference("statedim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("cubeZip",  new JoinChain("cubeZip", "cube-zip", "Zipcode thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "zipcode"));
 -            add(new TableReference("zipdim", "code"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("cubeCountry",  new JoinChain("cubeCountry", "cube-country", "country thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "countryid"));
 -            add(new TableReference("countrydim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("dim2chain", new JoinChain("dim2chain", "cube-testdim2", "testdim2 thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2"));
 -            add(new TableReference("testdim2", "id"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2big1"));
 -            add(new TableReference("testdim2", "bigid1"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2big2"));
 -            add(new TableReference("testdim2", "bigid2"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2bignew"));
 -            add(new TableReference("testdim2", "bigidnew"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("dim3chain", new JoinChain("dim3chain", "cube-testdim3", "cyclicdim thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2"));
 -            add(new TableReference("testdim2", "id"));
 -            add(new TableReference("testdim2", "testdim3id"));
 -            add(new TableReference("testdim3", "id"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2big1"));
 -            add(new TableReference("testdim2", "bigid1"));
 -            add(new TableReference("testdim2", "testdim3id"));
 -            add(new TableReference("testdim3", "id"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2big2"));
 -            add(new TableReference("testdim2", "bigid2"));
 -            add(new TableReference("testdim2", "testdim3id"));
 -            add(new TableReference("testdim3", "id"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2bignew"));
 -            add(new TableReference("testdim2", "bigidnew"));
 -            add(new TableReference("testdim2", "testdim3id"));
 -            add(new TableReference("testdim3", "id"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "testdim3id"));
 -            add(new TableReference("testdim3", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("dim4chain", new JoinChain("dim4chain", "cube-testdim3", "cyclicdim thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2"));
 -            add(new TableReference("testdim2", "id"));
 -            add(new TableReference("testdim2", "testdim3id"));
 -            add(new TableReference("testdim3", "id"));
 -            add(new TableReference("testdim3", "testdim4id"));
 -            add(new TableReference("testdim4", "id"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2big1"));
 -            add(new TableReference("testdim2", "bigid1"));
 -            add(new TableReference("testdim2", "testdim3id"));
 -            add(new TableReference("testdim3", "id"));
 -            add(new TableReference("testdim3", "testdim4id"));
 -            add(new TableReference("testdim4", "id"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2big2"));
 -            add(new TableReference("testdim2", "bigid2"));
 -            add(new TableReference("testdim2", "testdim3id"));
 -            add(new TableReference("testdim3", "id"));
 -            add(new TableReference("testdim3", "testdim4id"));
 -            add(new TableReference("testdim4", "id"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "dim2bignew"));
 -            add(new TableReference("testdim2", "bigidnew"));
 -            add(new TableReference("testdim2", "testdim3id"));
 -            add(new TableReference("testdim3", "id"));
 -            add(new TableReference("testdim3", "testdim4id"));
 -            add(new TableReference("testdim4", "id"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "testdim3id"));
 -            add(new TableReference("testdim3", "id"));
 -            add(new TableReference("testdim3", "testdim4id"));
 -            add(new TableReference("testdim4", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("cdimChain", new JoinChain("cdimChain", "cube-cyclicdim", "cyclicdim thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "cdim2"));
 -            add(new TableReference("cycledim1", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("unreachableDim_chain", new JoinChain("unreachableDim_chain", "cube-unreachableDim",
 -      "unreachableDim thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "urdimid"));
 -            add(new TableReference("unreachableDim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.put("cubeCountry",  new JoinChain("cubeCountry", "cube-country", "country thru cube") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference(cubeName, "countryid"));
 -            add(new TableReference("countrydim", "id"));
 -          }
 -        });
 -      }
 -    });
 -  }
 -  private void createBaseAndDerivedCubes(CubeMetastoreClient client)
 -    throws HiveException, ParseException, LensException {
 -    Set<CubeMeasure> cubeMeasures2 = new HashSet<>(cubeMeasures);
 -    Set<CubeDimAttribute> cubeDimensions2 = new HashSet<>(cubeDimensions);
 -    cubeMeasures2.add(new ColumnMeasure(new FieldSchema("msr11", "int", "first measure")));
 -    cubeMeasures2.add(new ColumnMeasure(new FieldSchema("msr12", "float", "second measure"), "Measure2", null, "SUM",
 -      "RS"));
 -    cubeMeasures2.add(new ColumnMeasure(new FieldSchema("msr13", "double", "third measure"), "Measure3", null, "MAX",
 -      null));
 -    cubeMeasures2.add(new ColumnMeasure(new FieldSchema("msr14", "bigint", "fourth measure"), "Measure4", null,
 -      "COUNT", null));
 -    cubeMeasures2.add(new ColumnMeasure(new FieldSchema("directMsr", "bigint", "fifth measure"), "Direct Measure",
 -      null, "SUM", null));
 -
 -    cubeDimensions2.add(new BaseDimAttribute(new FieldSchema("dim11", "string", "basedim")));
 -    cubeDimensions2.add(new ReferencedDimAttribute(new FieldSchema("dim12", "int", "ref dim"), "Dim2 refer",
 -      "dim2chain", "id", null, null, null)); // used as key in the chains
 -    cubeDimensions2.add(new ReferencedDimAttribute(new FieldSchema("dim22", "int", "ref dim"), "Dim2 refer",
 -      "dim2chain", "id", null, null, null)); // not used as key in the chains
 -    cubeDimensions2.add(new BaseDimAttribute(new FieldSchema("dim13", "string", "basedim")));
 -    cubeDimensions2.add(new BaseDimAttribute(new FieldSchema("userid", "int", "userid")));
 -    cubeDimensions2.add(new BaseDimAttribute(new FieldSchema("xuserid", "int", "userid")));
 -    cubeDimensions2.add(new BaseDimAttribute(new FieldSchema("yuserid", "int", "userid")));
 -    cubeDimensions2.add(new BaseDimAttribute(new FieldSchema("user_id_added_in_past", "int", "user_id_added_in_past")));
 -    cubeDimensions2.add(new BaseDimAttribute(new FieldSchema("user_id_added_far_future", "int",
 -        "user_id_added_far_future")));
 -    cubeDimensions2.add(new BaseDimAttribute(new FieldSchema("user_id_deprecated", "int", "user_id_deprecated")));
 -
 -    cubeDimensions2.add(new ReferencedDimAttribute(new FieldSchema("xsports", "array<string>", ""),
 -      "xuser sports", "xusersports", "name", null, null, null));
 -    cubeDimensions2.add(new ReferencedDimAttribute(new FieldSchema("ysports", "array<string>", ""),
 -      "yuser sports", "yusersports", "name", null, null, null));
 -    cubeDimensions2.add(new ReferencedDimAttribute(new FieldSchema("sports", "array<string>", ""),
 -      "user sports", "usersports", "name", null, null, null));
 -    cubeDimensions2.add(new ReferencedDimAttribute(new FieldSchema("sportids", "array<int>", ""),
 -      "user sports", "userInterestIds", "sport_id", null, null, null));
 -    cubeDimensions2.add(new ReferencedDimAttribute(new FieldSchema("statecountry", "string", ""),
 -      "state country", "cubestatecountry", "name", null, null, null));
 -    cubeDimensions2.add(new ReferencedDimAttribute(new FieldSchema("citycountry", "string", ""),
 -      "city country", "cubecitystatecountry", "name", null, null, null));
 -    List<ChainRefCol> refCols = new ArrayList<>();
 -    refCols.add(new ChainRefCol("cubeState", "countrycapital"));
 -    refCols.add(new ChainRefCol("cubeCityStateCountry", "capital"));
 -    cubeDimensions2.add(new ReferencedDimAttribute(new FieldSchema("cubeCountryCapital", "String", "ref dim"),
 -      "Country capital", refCols, null, null, null, null));
 -    Map<String, String> cubeProperties = new HashMap<>();
 -    cubeProperties.put(MetastoreUtil.getCubeTimedDimensionListKey(BASE_CUBE_NAME),
 -      "d_time,pt,it,et,test_time_dim,test_time_dim2");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "test_time_dim", "ttd");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "test_time_dim2", "ttd2");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "d_time", "dt");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "it", "it");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "et", "et");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_TO_PART_MAPPING_PFX + "processing_time", "pt");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_RELATION + "d_time", "processing_time+[-5 days,5 days]");
 -    cubeProperties.put(MetastoreConstants.TIMEDIM_RELATION + "processing_time", "test_time_dim+[-5 days,5 days]");
 -    cubeProperties.put(MetastoreConstants.CUBE_ALL_FIELDS_QUERIABLE, "false");
 -
 -    Map<String, JoinChain> joinChainMap = new HashMap<>();
 -    addCubeChains(joinChainMap, "basecube");
 -    // update new paths
 -    joinChainMap.get("dim2chain").addPath(new ArrayList<TableReference>() {
 -      {
 -        add(new TableReference("basecube", "dim12"));
 -        add(new TableReference("testdim2", "id"));
 -      }
 -    });
 -    joinChainMap.get("dim3chain").addPath(new ArrayList<TableReference>() {
 -      {
 -        add(new TableReference("basecube", "dim12"));
 -        add(new TableReference("testdim2", "id"));
 -        add(new TableReference("testdim2", "testdim3id"));
 -        add(new TableReference("testdim3", "id"));
 -      }
 -    });
 -    joinChainMap.get("dim4chain").addPath(new ArrayList<TableReference>() {
 -      {
 -        add(new TableReference("basecube", "dim12"));
 -        add(new TableReference("testdim2", "id"));
 -        add(new TableReference("testdim2", "testdim3id"));
 -        add(new TableReference("testdim3", "id"));
 -        add(new TableReference("testdim3", "testdim4id"));
 -        add(new TableReference("testdim4", "id"));
 -      }
 -    });
 -    Set<JoinChain> joinChains = Sets.newHashSet(joinChainMap.values());
 -    joinChains.add(new JoinChain("cityState", "city-state", "state thru city") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference("basecube", "cityid"));
 -            add(new TableReference("citydim", "id"));
 -            add(new TableReference("citydim", "stateid"));
 -            add(new TableReference("statedim", "id"));
 -          }
 -        });
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference("basecube", "cityid"));
 -            add(new TableReference("citydim", "id"));
 -            add(new TableReference("citydim", "statename"));
 -            add(new TableReference("statedim", "name"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.add(new JoinChain("cityZip", "city-zip", "zip thru city") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference("basecube", "cityid"));
 -            add(new TableReference("citydim", "id"));
 -            add(new TableReference("citydim", "zipcode"));
 -            add(new TableReference("zipdim", "code"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.add(new JoinChain("cubeStateCountry", "cube-state-country", "country through state") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference("basecube", "stateid"));
 -            add(new TableReference("statedim", "id"));
 -            add(new TableReference("statedim", "countryid"));
 -            add(new TableReference("countrydim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.add(new JoinChain("cubeCityStateCountry", "cube-city-state-country", "country through state thru city") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference("basecube", "cityid"));
 -            add(new TableReference("citydim", "id"));
 -            add(new TableReference("citydim", "stateid"));
 -            add(new TableReference("statedim", "id"));
 -            add(new TableReference("statedim", "countryid"));
 -            add(new TableReference("countrydim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.add(new JoinChain("userchain", "user-chain", "user chain") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference("basecube", "userid"));
 -            add(new TableReference("userdim", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.add(new JoinChain("user_id_added_far_future_chain", "user_id_added_far_future_chain",
 -        "user_id_added_far_future_chain") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference("basecube", "user_id_added_far_future"));
 -            add(new TableReference("userdim", "user_id_added_far_future"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.add(new JoinChain("userSports", "user-sports", "user sports") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference("basecube", "userid"));
 -            add(new TableReference("userdim", "id"));
 -            add(new TableReference("userdim", "id"));
 -            add(new TableReference("user_interests", "user_id", true));
 -            add(new TableReference("user_interests", "sport_id"));
 -            add(new TableReference("sports", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.add(new JoinChain("userInterestIds", "user-interestsIds", "user interest ids") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference("basecube", "userid"));
 -            add(new TableReference("userdim", "id"));
 -            add(new TableReference("userdim", "id"));
 -            add(new TableReference("user_interests", "user_id", true));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.add(new JoinChain("xuserSports", "xuser-sports", "xuser sports") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference("basecube", "xuserid"));
 -            add(new TableReference("userdim", "id"));
 -            add(new TableReference("userdim", "id"));
 -            add(new TableReference("user_interests", "user_id", true));
 -            add(new TableReference("user_interests", "sport_id"));
 -            add(new TableReference("sports", "id"));
 -          }
 -        });
 -      }
 -    });
 -    joinChains.add(new JoinChain("yuserSports", "user-sports", "user sports") {
 -      {
 -        addPath(new ArrayList<TableReference>() {
 -          {
 -            add(new TableReference("basecube", "yuserid"));
 -            add(new TableReference("userdim", "id"));
 -            add(new TableReference("userdim", "id"));
 -            add(new TableReference("user_interests", "user_id", true));
 -            add(new TableReference("user_interests", "sport_id"));
 -            add(new TableReference("sports", "id"));
 -          }
 -        });
 -      }
 -    });
 -
 -    // add ref dim through chain
 -    cubeDimensions2.add(
 -      new ReferencedDimAttribute(new FieldSchema("cityStateCapital", "string", "State's capital thru city"),
 -        "State's capital thru city", "cityState", "capital", null, null, null));
 -    Set<ExprColumn> baseExprs = new HashSet<>(exprs);
 -    baseExprs.add(new ExprColumn(new FieldSchema("substrsprorts", "String", "substr of sports"), "substr sports",
 -      "substr(sports, 10)"));
 -    baseExprs.add(new ExprColumn(new FieldSchema("xsports_abbr", "array<string>", ""),
 -      "xuser sports", "substr(xsports, 3)"));
 -    baseExprs.add(new ExprColumn(new FieldSchema("ysports_abbr", "array<string>", ""),
 -      "yuser sports", "substr(ysports, 3)"));
 -    baseExprs.add(new ExprColumn(new FieldSchema("sports_abbr", "array<string>", ""),
 -      "user sports", "substr(sports, 3)"));
 -    baseExprs.add(new ExprColumn(new FieldSchema("sportids_abbr", "array<string>", ""),
 -      "user sports", "case when sportids == 1 then 'CKT' when sportids == 2 then 'FTB' else 'NON' end"));
 -    baseExprs.add(new ExprColumn(new FieldSchema("directMsrExpr", "bigint", ""),
 -      "Direct Measure", new ExprSpec("directMsr + 0", null, null), new ExprSpec("msr13 + msr14", null, null)));
 -    client.createCube(BASE_CUBE_NAME, cubeMeasures2, cubeDimensions2, baseExprs, joinChains, cubeProperties);
 -
 -    Map<String, String> derivedProperties = new HashMap<>();
 -    derivedProperties.put(MetastoreConstants.CUBE_ALL_FIELDS_QUERIABLE, "true");
 -    Set<String> measures = new HashSet<>();
 -    measures.add("msr1");
 -    measures.add("msr9");
 -    measures.add("msr11");
 -    Set<String> dimensions = new HashSet<>();
 -    dimensions.add("dim1");
 -    dimensions.add("dim11");
 -    dimensions.add("d_time");
 -    client.createDerivedCube(BASE_CUBE_NAME, DERIVED_CUBE_NAME1, measures, dimensions, derivedProperties, 5L);
 -
 -    measures = new HashSet<>();
 -    measures.add("msr2");
 -    measures.add("msr12");
 -    measures.add("msr13");
 -    measures.add("msr14");
 -    measures.add("directmsr");
 -    dimensions = new HashSet<>();
 -    dimensions.add("cityid");
 -    dimensions.add("stateid");
 -    dimensions.add("userid");
 -    dimensions.add("xuserid");
 -    dimensions.add("yuserid");
 -    dimensions.add("dim1");
 -    dimensions.add("dim2");
 -    dimensions.add("dim2big1");
 -    dimensions.add("dim2big2");
 -    dimensions.add("dim2bignew");
 -    dimensions.add("dim11");
 -    dimensions.add("dim13");
 -    dimensions.add("dim12");
 -    dimensions.add("dim22");
 -    dimensions.add("d_time");
 -    dimensions.add("test_time_dim");
 -    dimensions.add("test_time_dim2");
 -    dimensions.add("test_time_dim_hour_id");
 -    dimensions.add("test_time_dim_day_id");
 -    dimensions.add("test_time_dim_hour_id2");
 -    dimensions.add("test_time_dim_day_id2");
 -    client.createDerivedCube(BASE_CUBE_NAME, DERIVED_CUBE_NAME2, measures, dimensions, derivedProperties, 10L);
 -    measures = new HashSet<>();
 -    measures.add("msr3");
 -    measures.add("msr13");
 -    dimensions = new HashSet<>();
 -    dimensions.add("dim1");
 -    dimensions.add("location");
 -    dimensions.add("d_time");
 -    dimensions.add("test_time_dim");
 -    dimensions.add("test_time_dim2");
 -    dimensions.add("test_time_dim_hour_id");
 -    dimensions.add("test_time_dim_day_id");
 -    dimensions.add("test_time_dim_hour_id2");
 -    dimensions.add("test_time_dim_day_id2");
 -    client.createDerivedCube(BASE_CUBE_NAME, DERIVED_CUBE_NAME3, measures, dimensions, derivedProperties, 20L);
 -
 -    // create base cube facts
 -    createBaseCubeFacts(client);
 -  }
 -
 -  private void createBaseCubeFacts(CubeMetastoreClient client) throws HiveException, LensException {
 -
 -    Map<String, Set<UpdatePeriod>> storageAggregatePeriods = new HashMap<String, Set<UpdatePeriod>>();
 -    Set<UpdatePeriod> updates = new HashSet<UpdatePeriod>();
 -    updates.add(MINUTELY);
 -    updates.add(HOURLY);
 -    updates.add(DAILY);
 -    updates.add(MONTHLY);
 -    updates.add(QUARTERLY);
 -    updates.add(YEARLY);
 -
 -    ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>();
 -    List<String> timePartCols = new ArrayList<String>();
 -    partCols.add(TestCubeMetastoreClient.getDatePartition());
 -    timePartCols.add(TestCubeMetastoreClient.getDatePartitionKey());
 -
 -    StorageTableDesc s1 = new StorageTableDesc();
 -    s1.setInputFormat(TextInputFormat.class.getCanonicalName());
 -    s1.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
 -    s1.setPartCols(partCols);
 -    s1.setTimePartCols(timePartCols);
 -
 -    StorageTableDesc s2 = new StorageTableDesc();
 -    s2.setInputFormat(TextInputFormat.class.getCanonicalName());
 -    s2.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
 -    ArrayList<FieldSchema> s2PartCols = new ArrayList<FieldSchema>();
 -    s2PartCols.add(new FieldSchema("ttd", serdeConstants.STRING_TYPE_NAME, "test date partition"));
 -    s2PartCols.add(new FieldSchema("ttd2", serdeConstants.STRING_TYPE_NAME, "test date partition"));
 -    s2.setPartCols(s2PartCols);
 -    s2.setTimePartCols(Arrays.asList("ttd", "ttd2"));
 -
 -    storageAggregatePeriods.put(c1, updates);
 -    storageAggregatePeriods.put(c2, updates);
 -    storageAggregatePeriods.put(c3, updates);
 -    storageAggregatePeriods.put(c4, updates);
 -
 -    Map<String, StorageTableDesc> storageTables = new HashMap<String, StorageTableDesc>();
 -    storageTables.put(c1, s1);
 -    storageTables.put(c4, s2);
 -    storageTables.put(c2, s1);
 -    storageTables.put(c3, s1);
 -
 -    String factName = "testFact1_BASE";
 -    List<FieldSchema> factColumns = new ArrayList<FieldSchema>(cubeMeasures.size());
 -    for (CubeMeasure measure : cubeMeasures) {
 -      factColumns.add(measure.getColumn());
 -    }
 -
 -    // add dimensions of the cube
 -    factColumns.add(new FieldSchema("d_time", "timestamp", "event time"));
 -    factColumns.add(new FieldSchema("processing_time", "timestamp", "processing time"));
 -    factColumns.add(new FieldSchema("zipcode", "int", "zip"));
 -    factColumns.add(new FieldSchema("cityid", "int", "city id"));
 -    factColumns.add(new FieldSchema("stateid", "int", "state id"));
 -    factColumns.add(new FieldSchema("userid", "int", "user id"));
 -    factColumns.add(new FieldSchema("xuserid", "int", "user id"));
 -    factColumns.add(new FieldSchema("yuserid", "int", "user id"));
 -    factColumns.add(new FieldSchema("dim1", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim11", "string", "base dim"));
 -    factColumns.add(new FieldSchema("test_time_dim_hour_id", "int", "time id"));
 -
 -    // create cube fact with materialized expressions
 -    client.createCubeFactTable(BASE_CUBE_NAME, factName, factColumns, storageAggregatePeriods, 5L,
 -      factValidityProperties, storageTables);
 -
 -    factName = "testFact5_BASE";
 -    factColumns = new ArrayList<>(cubeMeasures.size());
 -    for (CubeMeasure measure : cubeMeasures) {
 -      factColumns.add(measure.getColumn());
 -    }
 -
 -    // add dimensions of the cube
 -    factColumns.add(new FieldSchema("d_time", "timestamp", "event time"));
 -    factColumns.add(new FieldSchema("processing_time", "timestamp", "processing time"));
 -    factColumns.add(new FieldSchema("dim1", "string", "base dim"));
 -    factColumns.add(new FieldSchema("booleancut", "boolean", "expr dim"));
 -
 -    // create cube fact
 -    client.createCubeFactTable(BASE_CUBE_NAME, factName, factColumns, storageAggregatePeriods, 150L,
 -      factValidityProperties, storageTables);
 -
 -    // create fact only with extra measures
 -    factName = "testFact2_BASE";
 -    factColumns = new ArrayList<FieldSchema>();
 -    factColumns.add(new FieldSchema("msr12", "float", "second measure"));
 -
 -    // add dimensions of the cube
 -    factColumns.add(new FieldSchema("d_time", "timestamp", "event time"));
 -    factColumns.add(new FieldSchema("processing_time", "timestamp", "processing time"));
 -    factColumns.add(new FieldSchema("dim1", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim11", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim2", "int", "dim2 id"));
 -    factColumns.add(new FieldSchema("userid", "int", "user id"));
 -    factColumns.add(new FieldSchema("xuserid", "int", "user id"));
 -    factColumns.add(new FieldSchema("yuserid", "int", "user id"));
 -    // create cube fact
 -    client.createCubeFactTable(BASE_CUBE_NAME, factName, factColumns, storageAggregatePeriods, 5L,
 -      factValidityProperties, storageTables);
 -    Map<String, String> properties = Maps.newHashMap(factValidityProperties);
 -    properties.put(MetastoreConstants.FACT_ABSOLUTE_END_TIME, DateUtil.relativeToAbsolute("now.day - 2 days"));
 -    properties.put(MetastoreConstants.FACT_ABSOLUTE_START_TIME, DateUtil.relativeToAbsolute("now.day - 3 days"));
 -    client.createCubeFactTable(BASE_CUBE_NAME, "testfact_deprecated", factColumns, storageAggregatePeriods, 5L,
 -      properties, storageTables);
 -
 -    // create fact only with extra measures
 -    factName = "testFact3_BASE";
 -    factColumns = new ArrayList<FieldSchema>();
 -    factColumns.add(new FieldSchema("msr13", "double", "third measure"));
 -    factColumns.add(new FieldSchema("msr14", "bigint", "fourth measure"));
 -
 -    // add dimensions of the cube
 -    factColumns.add(new FieldSchema("d_time", "timestamp", "event time"));
 -    factColumns.add(new FieldSchema("processing_time", "timestamp", "processing time"));
 -    factColumns.add(new FieldSchema("dim1", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim11", "string", "base dim"));
 -
 -    // create cube fact
 -    client.createCubeFactTable(BASE_CUBE_NAME, factName, factColumns, storageAggregatePeriods, 5L,
 -      factValidityProperties, storageTables);
 -
 -    // create fact with materialized expression
 -    factName = "testFact6_BASE";
 -    factColumns = new ArrayList<>();
 -    factColumns.add(new FieldSchema("msr13", "double", "third measure"));
 -    factColumns.add(new FieldSchema("msr14", "bigint", "fourth measure"));
 -
 -    // add dimensions of the cube
 -    factColumns.add(new FieldSchema("d_time", "timestamp", "event time"));
 -    factColumns.add(new FieldSchema("processing_time", "timestamp", "processing time"));
 -    factColumns.add(new FieldSchema("dim1", "string", "base dim"));
 -    factColumns.add(new FieldSchema("booleancut", "boolean", "expr dim"));
 -
 -    // create cube fact
 -    client.createCubeFactTable(BASE_CUBE_NAME, factName, factColumns, storageAggregatePeriods, 150L,
 -      factValidityProperties, storageTables);
 -
 -    // create raw fact only with extra measures
 -    factName = "testFact2_RAW_BASE";
 -    factColumns = new ArrayList<FieldSchema>();
 -    factColumns.add(new FieldSchema("msr11", "int", "first measure"));
 -    factColumns.add(new FieldSchema("msr12", "float", "second measure"));
 -    factColumns.add(new FieldSchema("msr9", "bigint", "ninth measure"));
 -
 -    // add dimensions of the cube
 -    factColumns.add(new FieldSchema("d_time", "timestamp", "event time"));
 -    factColumns.add(new FieldSchema("processing_time", "timestamp", "processing time"));
 -    factColumns.add(new FieldSchema("dim1", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim11", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim13", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim12", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim22", "string", "base dim"));
 -    factColumns.add(new FieldSchema("cityid", "int", "city id"));
 -
 -    storageAggregatePeriods = new HashMap<String, Set<UpdatePeriod>>();
 -    updates = new HashSet<UpdatePeriod>();
 -    updates.add(HOURLY);
 -    storageAggregatePeriods.put(c1, updates);
 -
 -    storageTables = new HashMap<String, StorageTableDesc>();
 -    storageTables.put(c1, s1);
 -
 -    // create cube fact
 -    properties.clear();
 -    properties.putAll(factValidityProperties);
 -    properties.put(MetastoreConstants.FACT_AGGREGATED_PROPERTY, "false");
 -    properties.put(MetastoreConstants.FACT_DATA_COMPLETENESS_TAG, "f2");
 -
 -    client.createCubeFactTable(BASE_CUBE_NAME, factName, factColumns, storageAggregatePeriods, 100L, properties,
 -      storageTables);
 -
 -    // create raw fact only with extra measures
 -    factName = "testFact3_RAW_BASE";
 -    factColumns = new ArrayList<FieldSchema>();
 -    factColumns.add(new FieldSchema("msr13", "double", "third measure"));
 -    factColumns.add(new FieldSchema("msr14", "bigint", "fourth measure"));
 -
 -    // add dimensions of the cube
 -    factColumns.add(new FieldSchema("d_time", "timestamp", "event time"));
 -    factColumns.add(new FieldSchema("processing_time", "timestamp", "processing time"));
 -    factColumns.add(new FieldSchema("dim1", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim11", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim12", "string", "base dim"));
 -
 -    storageAggregatePeriods = new HashMap<String, Set<UpdatePeriod>>();
 -    updates = new HashSet<UpdatePeriod>();
 -    updates.add(HOURLY);
 -    storageAggregatePeriods.put(c1, updates);
 -
 -    storageTables = new HashMap<String, StorageTableDesc>();
 -    storageTables.put(c1, s1);
 -    properties.put(MetastoreConstants.FACT_COL_START_TIME_PFX.concat("user_id_added_in_past"), "2016-01-01");
 -    properties.put(MetastoreConstants.FACT_COL_END_TIME_PFX.concat("user_id_deprecated"), "2016-01-01");
 -    properties.put(MetastoreConstants.FACT_COL_START_TIME_PFX.concat("user_id_added_far_future"), "2099-01-01");
 -    client.createCubeFactTable(BASE_CUBE_NAME, factName, factColumns, storageAggregatePeriods, 100L, properties,
 -      storageTables);
 -
 -    factName = "testFact4_RAW_BASE";
 -    factColumns = new ArrayList<FieldSchema>();
 -    factColumns.add(new FieldSchema("msr13", "double", "third measure"));
 -    factColumns.add(new FieldSchema("msr14", "bigint", "fourth measure"));
 -
 -    // add dimensions of the cube
 -    factColumns.add(new FieldSchema("d_time", "timestamp", "event time"));
 -    factColumns.add(new FieldSchema("processing_time", "timestamp", "processing time"));
 -    factColumns.add(new FieldSchema("dim1", "string", "base dim"));
 -    factColumns.add(new FieldSchema("user_id_added_in_past", "int", "user id"));
 -    factColumns.add(new FieldSchema("user_id_added_far_future", "int", "user id"));
 -    factColumns.add(new FieldSchema("user_id_deprecated", "int", "user id"));
 -
 -    storageTables = new HashMap<String, StorageTableDesc>();
 -    storageTables.put(c1, s1);
 -    properties.put(MetastoreConstants.FACT_COL_START_TIME_PFX.concat("user_id_added_in_past"), "2016-01-01");
 -    properties.put(MetastoreConstants.FACT_COL_END_TIME_PFX.concat("user_id_deprecated"), "2016-01-01");
 -    properties.put(MetastoreConstants.FACT_COL_START_TIME_PFX.concat("user_id_added_far_future"), "2099-01-01");
 -    client.createCubeFactTable(BASE_CUBE_NAME, factName, factColumns, storageAggregatePeriods, 100L, properties,
 -        storageTables);
 -
 -    factName = "testFact5_RAW_BASE";
 -    factColumns = new ArrayList<FieldSchema>();
 -    factColumns.add(new FieldSchema("msr9", "bigint", "ninth measure"));
 -
 -    // add dimensions of the cube
 -    factColumns.add(new FieldSchema("d_time", "timestamp", "event time"));
 -    factColumns.add(new FieldSchema("processing_time", "timestamp", "processing time"));
 -    factColumns.add(new FieldSchema("dim1", "string", "base dim"));
 -
 -    properties.clear();
 -    properties.putAll(factValidityProperties);
 -    properties.put(MetastoreConstants.FACT_AGGREGATED_PROPERTY, "false");
 -    properties.put(MetastoreConstants.FACT_DATA_COMPLETENESS_TAG, "f2");
 -    client.createCubeFactTable(BASE_CUBE_NAME, factName, factColumns, storageAggregatePeriods, 100L, properties,
 -            storageTables);
 -
 -    CubeFactTable fact = client.getFactTable(factName);
 -    // Add all hourly partitions for two days
 -    Calendar cal = Calendar.getInstance();
 -    cal.setTime(TWODAYS_BACK);
 -    Date temp = cal.getTime();
 -    while (!(temp.after(NOW))) {
 -      Map<String, Date> timeParts = new HashMap<String, Date>();
 -      timeParts.put("dt", temp);
 -      StoragePartitionDesc sPartSpec = new StoragePartitionDesc(fact.getName(), timeParts, null, HOURLY);
 -      client.addPartition(sPartSpec, c1, CubeTableType.FACT);
 -      cal.add(HOUR_OF_DAY, 1);
 -      temp = cal.getTime();
 -    }
 -  }
 -
 -  private void createCubeContinuousFact(CubeMetastoreClient client) throws Exception {
 -    // create continuous raw fact only with extra measures
 -    String factName = "testFact_CONTINUOUS";
 -    List<FieldSchema> factColumns = new ArrayList<FieldSchema>();
 -    factColumns.add(new FieldSchema("msr11", "double", "third measure"));
 -    factColumns.add(new FieldSchema("msr15", "int", "fifteenth measure"));
 -
 -    // add dimensions of the cube
 -    factColumns.add(new FieldSchema("d_time", "timestamp", "event time"));
 -    factColumns.add(new FieldSchema("processing_time", "timestamp", "processing time"));
 -    factColumns.add(new FieldSchema("dim1", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim11", "string", "base dim"));
 -    factColumns.add(new FieldSchema("dim12", "string", "base dim"));
 -
 -    Map<String, Set<UpdatePeriod>> storageAggregatePeriods = new HashMap<String, Set<UpdatePeriod>>();
 -    Set<UpdatePeriod> updates = new HashSet<UpdatePeriod>();
 -    updates.add(CONTINUOUS);
 -    storageAggregatePeriods.put(c0, updates);
 -
 -    StorageTableDesc s0 = new StorageTableDesc();
 -    s0.setInputFormat(TextInputFormat.class.getCanonicalName());
 -    s0.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
 -
 -    Map<String, StorageTableDesc> storageTables = new HashMap<String, StorageTableDesc>();
 -    storageTables.put(c0, s0);
 -    Map<String, String> properties = Maps.newHashMap(factValidityProperties);
 -    properties.put(MetastoreConstants.FACT_ABSOLUTE_START_TIME, DateUtil.relativeToAbsolute("now.day - 3 days"));
 -
 -    client.createCubeFactTable(TEST_CUBE_NAME, factName, factColumns, storageAggregatePeriods, 100L, properties,
 -      storageTables);
 -  }
--
 -  private void createCubeFact(CubeMetastoreClient client) throws Exception {
 +  private void assertTestFactTimelineClass(CubeMetastoreClient client) throws Exception {
      String factName = "testFact";
 -    List<FieldSchema> factColumns = new ArrayList<FieldSchema>(cubeMeasures.size());
 -    for (CubeMeasure measure : cubeMeasures) {
 -      if (!measure.getColumn().getName().equals("msr15")) { //do not add msr15
 -        factColumns.add(measure.getColumn());
 -      }
 -    }
 -    factColumns.add(new FieldSchema("msr5", "double", "msr5"));
 -
 -    // add dimensions of the cube
 -    factColumns.add(new FieldSchema("zipcode", "int", "zip"));
 -    factColumns.add(new FieldSchema("cityid", "int", "city id"));
 -    factColumns.add(new FieldSchema("cityid1", "int", "city id"));
 -    factColumns.add(new FieldSchema("stateid", "int", "city id"));
 -    factColumns.add(new FieldSchema("test_time_dim_day_id", "int", "time id"));
 -    factColumns.add(new FieldSchema("test_time_dim_day_id2", "int", "time id"));
 -    factColumns.add(new FieldSchema("ambigdim1", "string", "used in" + " testColumnAmbiguity"));
 -
 -    Map<String, Set<UpdatePeriod>> storageAggregatePeriods = new HashMap<String, Set<UpdatePeriod>>();
 -    Set<UpdatePeriod> updates = new HashSet<UpdatePeriod>();
 -    updates.add(MINUTELY);
 -    updates.add(HOURLY);
 -    updates.add(DAILY);
 -    updates.add(MONTHLY);
 -    updates.add(QUARTERLY);
 -    updates.add(YEARLY);
 -
 -    ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>();
 -    List<String> timePartCols = new ArrayList<String>();
 -    partCols.add(TestCubeMetastoreClient.getDatePartition());
 -    timePartCols.add(TestCubeMetastoreClient.getDatePartitionKey());
 -
 -    StorageTableDesc s1 = new StorageTableDesc();
 -    s1.setInputFormat(TextInputFormat.class.getCanonicalName());
 -    s1.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
 -    s1.setPartCols(partCols);
 -    s1.setTimePartCols(timePartCols);
 -
 -    StorageTableDesc s2 = new StorageTableDesc();
 -    s2.setInputFormat(TextInputFormat.class.getCanonicalName());
 -    s2.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
 -    ArrayList<FieldSchema> s2PartCols = new ArrayList<FieldSchema>();
 -    s2PartCols.add(new FieldSchema("ttd", serdeConstants.STRING_TYPE_NAME, "test date partition"));
 -    s2PartCols.add(new FieldSchema("ttd2", serdeConstants.STRING_TYPE_NAME, "test date partition"));
 -    s2.setPartCols(s2PartCols);
 -    s2.setTimePartCols(Arrays.asList("ttd", "ttd2"));
 -
 -    StorageTableDesc s3 = new StorageTableDesc();
 -    s3.setInputFormat(TextInputFormat.class.getCanonicalName());
 -    s3.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
 -    s3.setPartCols(partCols);
 -    s3.setTimePartCols(timePartCols);
 -    s3.getTblProps().put(MetastoreUtil.getStoragetableStartTimesKey(), "now.day - 90 days");
 -    s3.getTblProps().put(MetastoreUtil.getStoragetableEndTimesKey(), "now.day - 10 days");
 -
 -    StorageTableDesc s5 = new StorageTableDesc();
 -    s5.setInputFormat(TextInputFormat.class.getCanonicalName());
 -    s5.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
 -    s5.setPartCols(partCols);
 -    s5.setTimePartCols(timePartCols);
 -    s5.getTblProps().put(MetastoreUtil.getStoragetableStartTimesKey(), "now.day - 10 days");
 -
 -    storageAggregatePeriods.put(c1, updates);
 -    storageAggregatePeriods.put(c2, updates);
 -    storageAggregatePeriods.put(c3, updates);
 -    storageAggregatePeriods.put(c4, updates);
 -    storageAggregatePeriods.put(c5, updates);
 -
 -    Map<String, StorageTableDesc> storageTables = new HashMap<String, StorageTableDesc>();
 -    storageTables.put(c1, s1);
 -    storageTables.put(c4, s2);
 -    storageTables.put(c2, s1);
 -    storageTables.put(c3, s3);
 -    storageTables.put(c5, s5);
 -
 -    //add storage with continuous update period
 -    updates.add(CONTINUOUS);
 -    storageAggregatePeriods.put(c0, updates);
 -    StorageTableDesc s0 = new StorageTableDesc();
 -    s0.setInputFormat(TextInputFormat.class.getCanonicalName());
 -    s0.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
 -    storageTables.put(c0, s0);
 -
 -    // create cube fact
 -    client.createCubeFactTable(TEST_CUBE_NAME, factName, factColumns, storageAggregatePeriods, 5L,
 -      factValidityProperties, storageTables);
 +
      client.getTimelines(factName, c1, null, null);
      client.getTimelines(factName, c4, null, null);
 +
      client.clearHiveTableCache();
 +
      CubeFactTable fact = client.getFactTable(factName);
      Table table = client.getTable(MetastoreUtil.getStorageTableName(fact.getName(), Storage.getPrefix(c1)));
      assertEquals(table.getParameters().get(MetastoreUtil.getPartitionTimelineCachePresenceKey()), "true");

http://git-wip-us.apache.org/repos/asf/lens/blob/8868b063/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
----------------------------------------------------------------------
diff --cc lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
index abaae5b,24660e1..194b380
--- a/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
@@@ -226,10 -232,24 +226,8 @@@ public class CubeMetastoreServiceImpl e
     */
    @Override
    public void createDimensionTable(LensSessionHandle sessionid, XDimensionTable xDimTable) throws LensException {
--    String dimTblName = xDimTable.getTableName();
 -    List<FieldSchema> columns = JAXBUtils.fieldSchemaListFromColumns(xDimTable.getColumns());
 -    Map<String, UpdatePeriod> updatePeriodMap =
 -      JAXBUtils.dumpPeriodsFromStorageTables(xDimTable.getStorageTables());
 -
 -    Map<String, String> properties = JAXBUtils.mapFromXProperties(xDimTable.getProperties());
 -    Map<String, StorageTableDesc> storageDesc = JAXBUtils.tableDescPrefixMapFromXStorageTables(
 -      xDimTable.getStorageTables());
--
      try (SessionContext ignored = new SessionContext(sessionid)){
 -      log.info("# Columns: " + columns);
 -      getClient(sessionid).createCubeDimensionTable(xDimTable.getDimensionName(),
 -        dimTblName,
 -        columns,
 -        xDimTable.getWeight(),
 -        updatePeriodMap,
 -        properties,
 -        storageDesc);
 +      getClient(sessionid).createCubeDimensionTable(xDimTable);
        log.info("Dimension Table created " + xDimTable.getTableName());
      }
    }


[04/16] lens git commit: LENS-1373 : Specify chain of comparators for query comparator config

Posted by pr...@apache.org.
LENS-1373 : Specify chain of comparators for query comparator config


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

Branch: refs/heads/lens-1381
Commit: 29568fa3999642d017977b5dda3d7e08bee91337
Parents: 5ac6880
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Thu Jan 19 10:11:27 2017 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Thu Jan 19 10:11:27 2017 +0530

----------------------------------------------------------------------
 .../server/query/QueryExecutionServiceImpl.java |   3 +-
 .../src/main/resources/lensserver-default.xml   |  11 ++
 src/site/apt/admin/config.apt                   | 128 ++++++++++---------
 3 files changed, 77 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/29568fa3/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 c76ad24..c6fbeda 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
@@ -404,8 +404,7 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
   private void loadQueryComparator() throws LensException {
     try {
       Class<?>[] classes = conf.getClasses(QUERY_COMPARATOR_CLASSES,
-        MoreRetriesFirstComparator.class, QueryPriorityComparator.class,
-        FIFOQueryComparator.class, QueryCostComparator.class);
+        MoreRetriesFirstComparator.class, QueryPriorityComparator.class, FIFOQueryComparator.class);
       List<Comparator<QueryContext>> comparators = Lists.newArrayList();
       for (Class<?> clazz: classes) {
         comparators.add(clazz.asSubclass(QueryComparator.class).newInstance());

http://git-wip-us.apache.org/repos/asf/lens/blob/29568fa3/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 261fa52..28b1db2 100644
--- a/lens-server/src/main/resources/lensserver-default.xml
+++ b/lens-server/src/main/resources/lensserver-default.xml
@@ -942,6 +942,17 @@
     <value>org.apache.lens.server.api.query.cost.FactPartitionBasedQueryCost$Parser</value>
     <description>The Query cost parser class. Default query cost class used is FactPartitionBasedQueryCost</description>
   </property>
+
+  <property>
+    <name>lens.server.query.comparator.classes</name>
+    <value>org.apache.lens.server.api.query.comparators.MoreRetriesFirstComparator,org.apache.lens.server.api.query.comparators.QueryPriorityComparator,org.apache.lens.server.api.query.comparators.FIFOQueryComparator</value>
+    <description>The Query cost comparator chain. Queries are compared in this order. To compare queries q1 and q2,
+    first number of retries are considered. The one with more retries is placed first in the queue. If those are same,
+    then their priorities are considered, with higher priorities coming before lower ones. If those are also same,
+    then their submission times are considered. The query that was submitted first is placed first.
+    </description>
+  </property>
+
   <property>
     <name>lens.cube.metastore.enable.datacompleteness.check</name>
     <value>false</value>

http://git-wip-us.apache.org/repos/asf/lens/blob/29568fa3/src/site/apt/admin/config.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/admin/config.apt b/src/site/apt/admin/config.apt
index eb35ae3..8720c82 100644
--- a/src/site/apt/admin/config.apt
+++ b/src/site/apt/admin/config.apt
@@ -163,132 +163,134 @@ Lens server configuration
 *--+--+---+--+
 |68|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.|
 *--+--+---+--+
-|69|lens.server.query.cost.parser.class|org.apache.lens.server.api.query.cost.FactPartitionBasedQueryCost$Parser|The Query cost parser class. Default query cost class used is FactPartitionBasedQueryCost|
+|69|lens.server.query.comparator.classes|org.apache.lens.server.api.query.comparators.MoreRetriesFirstComparator,org.apache.lens.server.api.query.comparators.QueryPriorityComparator,org.apache.lens.server.api.query.comparators.FIFOQueryComparator|The Query cost comparator chain. Queries are compared in this order. To compare queries q1 and q2, first number of retries are considered. The one with more retries is placed first in the queue. If those are same, then their priorities are considered, with higher priorities coming before lower ones. If those are also same, then their submission times are considered. The query that was submitted first is placed first.|
 *--+--+---+--+
-|70|lens.server.query.expiry.check.interval.millis|60000|The interval(milliseconds) with which query expiry will run periodically. Default is 1 minute. The value needs to be much lower than lens.query.timeout.millis. If the final deployment values of query timeout can be smaller, then reduce this value to be much lower.|
+|70|lens.server.query.cost.parser.class|org.apache.lens.server.api.query.cost.FactPartitionBasedQueryCost$Parser|The Query cost parser class. Default query cost class used is FactPartitionBasedQueryCost|
 *--+--+---+--+
-|71|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.|
+|71|lens.server.query.expiry.check.interval.millis|60000|The interval(milliseconds) with which query expiry will run periodically. Default is 1 minute. The value needs to be much lower than lens.query.timeout.millis. If the final deployment values of query timeout can be smaller, then reduce this value to be much lower.|
 *--+--+---+--+
-|72|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.\ |
+|72|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.|
+*--+--+---+--+
+|73|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 :)                                                                                                |
 *--+--+---+--+
-|73|lens.server.query.resultset.retention|1 day|Lens query resultset retention period. Default 1 day|
+|74|lens.server.query.resultset.retention|1 day|Lens query resultset retention period. Default 1 day|
 *--+--+---+--+
-|74|lens.server.query.service.impl|org.apache.lens.server.query.QueryExecutionServiceImpl|Implementation class for query execution service|
+|75|lens.server.query.service.impl|org.apache.lens.server.query.QueryExecutionServiceImpl|Implementation class for query execution service|
 *--+--+---+--+
-|75|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|
+|76|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|
 *--+--+---+--+
-|76|lens.server.query.ws.resource.impl|org.apache.lens.server.query.QueryServiceResource|Implementation class for Query Resource|
+|77|lens.server.query.ws.resource.impl|org.apache.lens.server.query.QueryServiceResource|Implementation class for Query Resource|
 *--+--+---+--+
-|77|lens.server.querypurger.sleep.interval|10000|The interval(milliseconds) with which purger to run periodically. Default 10 sec.|
+|78|lens.server.querypurger.sleep.interval|10000|The interval(milliseconds) with which purger to run periodically. Default 10 sec.|
 *--+--+---+--+
-|78|lens.server.quota.service.impl|org.apache.lens.server.quota.QuotaServiceImpl|Implementation class for quota service|
+|79|lens.server.quota.service.impl|org.apache.lens.server.quota.QuotaServiceImpl|Implementation class for quota service|
 *--+--+---+--+
-|79|lens.server.quota.ws.resource.impl|org.apache.lens.server.quota.QuotaResource|Implementation class for Quota Resource|
+|80|lens.server.quota.ws.resource.impl|org.apache.lens.server.quota.QuotaResource|Implementation class for Quota Resource|
 *--+--+---+--+
-|80|lens.server.requestlogger.ws.filter.impl|org.apache.lens.server.LensRequestLoggingFilter|Implementation class for Request logging Filter|
+|81|lens.server.requestlogger.ws.filter.impl|org.apache.lens.server.LensRequestLoggingFilter|Implementation class for Request logging Filter|
 *--+--+---+--+
-|81|lens.server.resultset.purge.enabled|false|Whether to purge the query results|
+|82|lens.server.resultset.purge.enabled|false|Whether to purge the query results|
 *--+--+---+--+
-|82|lens.server.resultsetpurger.sleep.interval.secs|3600|Periodicity for Query result purger runs. Default 1 hour.|
+|83|lens.server.resultsetpurger.sleep.interval.secs|3600|Periodicity for Query result purger runs. Default 1 hour.|
 *--+--+---+--+
-|83|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.|
+|84|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.|
 *--+--+---+--+
-|84|lens.server.savedquery.list.default.count|20|Key denoting the default fetch value of saved query list api.|
+|85|lens.server.savedquery.list.default.count|20|Key denoting the default fetch value of saved query list api.|
 *--+--+---+--+
-|85|lens.server.savedquery.list.default.offset|0|Key denoting the default start value of saved query list api.|
+|86|lens.server.savedquery.list.default.offset|0|Key denoting the default start value of saved query list api.|
 *--+--+---+--+
-|86|lens.server.savedquery.service.impl|org.apache.lens.server.query.save.SavedQueryServiceImpl|Implementation class for saved query service|
+|87|lens.server.savedquery.service.impl|org.apache.lens.server.query.save.SavedQueryServiceImpl|Implementation class for saved query service|
 *--+--+---+--+
-|87|lens.server.savedquery.ws.resource.impl|org.apache.lens.server.query.save.SavedQueryResource|Implementation class for Saved query Resource|
+|88|lens.server.savedquery.ws.resource.impl|org.apache.lens.server.query.save.SavedQueryResource|Implementation class for Saved query Resource|
 *--+--+---+--+
-|88|lens.server.scheduler.instance.waiting.thread.interval.millis|300000|Thread interval for checking the waiting instances in milliseconds|
+|89|lens.server.scheduler.instance.waiting.thread.interval.millis|300000|Thread interval for checking the waiting instances in milliseconds|
 *--+--+---+--+
-|89|lens.server.scheduler.max.job.per.user|-1|Maximum number of jobs that can be scheduled by a single user. If the number is less than zero, then there is no restriction on the number of jobs scheduled.|
+|90|lens.server.scheduler.max.job.per.user|-1|Maximum number of jobs that can be scheduled by a single user. If the number is less than zero, then there is no restriction on the number of jobs scheduled.|
 *--+--+---+--+
-|90|lens.server.scheduler.service.impl|org.apache.lens.server.scheduler.SchedulerServiceImpl|Implementation class for query scheduler service|
+|91|lens.server.scheduler.service.impl|org.apache.lens.server.scheduler.SchedulerServiceImpl|Implementation class for query scheduler service|
 *--+--+---+--+
-|91|lens.server.scheduler.store.class|org.apache.lens.server.scheduler.SchedulerDAO$SchedulerHsqlDBStore|A subclass of SchedulerDBStore class used for storing scheduler related information.|
+|92|lens.server.scheduler.store.class|org.apache.lens.server.scheduler.SchedulerDAO$SchedulerHsqlDBStore|A subclass of SchedulerDBStore class used for storing scheduler related information.|
 *--+--+---+--+
-|92|lens.server.scheduler.ws.resource.impl|org.apache.lens.server.scheduler.ScheduleResource|Implementation class for query scheduler resource|
+|93|lens.server.scheduler.ws.resource.impl|org.apache.lens.server.scheduler.ScheduleResource|Implementation class for query scheduler resource|
 *--+--+---+--+
-|93|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.|
+|94|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.|
 *--+--+---+--+
-|94|lens.server.serverMode.ws.filter.impl|org.apache.lens.server.ServerModeFilter|Implementation class for ServerMode Filter|
+|95|lens.server.serverMode.ws.filter.impl|org.apache.lens.server.ServerModeFilter|Implementation class for ServerMode Filter|
 *--+--+---+--+
-|95|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.|
+|96|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.|
 *--+--+---+--+
-|96|lens.server.servicenames|session,alarm,query,savedquery,metastore,scheduler,quota|These services would be started in the specified order when lens-server starts up|
+|97|lens.server.servicenames|session,alarm,query,savedquery,metastore,scheduler,quota|These services would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|97|lens.server.session.expiry.service.interval.secs|3600|Interval at which lens session expiry service runs|
+|98|lens.server.session.expiry.service.interval.secs|3600|Interval at which lens session expiry service runs|
 *--+--+---+--+
-|98|lens.server.session.service.impl|org.apache.lens.server.session.HiveSessionService|Implementation class for session service|
+|99|lens.server.session.service.impl|org.apache.lens.server.session.HiveSessionService|Implementation class for session service|
 *--+--+---+--+
-|99|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.|
+|100|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.|
 *--+--+---+--+
-|100|lens.server.session.ws.resource.impl|org.apache.lens.server.session.SessionResource|Implementation class for Session Resource|
+|101|lens.server.session.ws.resource.impl|org.apache.lens.server.session.SessionResource|Implementation class for Session Resource|
 *--+--+---+--+
-|101|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.|
+|102|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.|
 *--+--+---+--+
-|102|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.|
+|103|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.|
 *--+--+---+--+
-|103|lens.server.state.persistence.interval.millis|300000|Lens server state persistence time interval in milliseconds|
+|104|lens.server.state.persistence.interval.millis|300000|Lens server state persistence time interval in milliseconds|
 *--+--+---+--+
-|104|lens.server.statistics.db|lensstats|Database to which statistics tables are created and partitions are added.|
+|105|lens.server.statistics.db|lensstats|Database to which statistics tables are created and partitions are added.|
 *--+--+---+--+
-|105|lens.server.statistics.log.rollover.interval|3600000|Default rate which log statistics store scans for rollups in milliseconds.|
+|106|lens.server.statistics.log.rollover.interval|3600000|Default rate which log statistics store scans for rollups in milliseconds.|
 *--+--+---+--+
-|106|lens.server.statistics.store.class|org.apache.lens.server.stats.store.log.LogStatisticsStore|Default implementation of class used to persist Lens Statistics.|
+|107|lens.server.statistics.store.class|org.apache.lens.server.stats.store.log.LogStatisticsStore|Default implementation of class used to persist Lens Statistics.|
 *--+--+---+--+
-|107|lens.server.statistics.warehouse.dir|file:///tmp/lens/statistics/warehouse|Default top level location where stats are moved by the log statistics store.|
+|108|lens.server.statistics.warehouse.dir|file:///tmp/lens/statistics/warehouse|Default top level location where stats are moved by the log statistics store.|
 *--+--+---+--+
-|108|lens.server.status.update.exponential.wait.millis|30000|Number of millis that would grow exponentially for next update, incase of transient failures.|
+|109|lens.server.status.update.exponential.wait.millis|30000|Number of millis that would grow exponentially for next update, incase of transient failures.|
 *--+--+---+--+
-|109|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.|
+|110|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.|
 *--+--+---+--+
-|110|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.|
+|111|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.|
 *--+--+---+--+
-|111|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.|
+|112|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.|
 *--+--+---+--+
-|112|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|
+|113|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|
 *--+--+---+--+
-|113|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.|
+|114|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.|
 *--+--+---+--+
-|114|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.|
+|115|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.|
 *--+--+---+--+
-|115|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.|
+|116|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.|
 *--+--+---+--+
-|116|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...|
+|117|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...|
 *--+--+---+--+
-|117|lens.server.user.resolver.ldap.bind.password| |Required for LDAP_BACKED_DATABASE user resolvers. ldap password for admin binding above|
+|118|lens.server.user.resolver.ldap.bind.password| |Required for LDAP_BACKED_DATABASE user resolvers. ldap password for admin binding above|
 *--+--+---+--+
-|118|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.|
+|119|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.|
 *--+--+---+--+
-|119|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.|
+|120|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.|
 *--+--+---+--+
-|120|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|
+|121|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|
 *--+--+---+--+
-|121|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.|
+|122|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.|
 *--+--+---+--+
-|122|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...|
+|123|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...|
 *--+--+---+--+
-|123|lens.server.user.resolver.ldap.search.filter|(&(objectClass=user)(sAMAccountName=%s))|Required for LDAP_BACKED_DATABASE user resolvers. filter pattern for ldap search|
+|124|lens.server.user.resolver.ldap.search.filter|(&(objectClass=user)(sAMAccountName=%s))|Required for LDAP_BACKED_DATABASE user resolvers. filter pattern for ldap search|
 *--+--+---+--+
-|124|lens.server.user.resolver.ldap.url| |Required for LDAP_BACKED_DATABASE user resolvers. ldap url to connect to.|
+|125|lens.server.user.resolver.ldap.url| |Required for LDAP_BACKED_DATABASE user resolvers. ldap url to connect to.|
 *--+--+---+--+
-|125|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|
+|126|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|
 *--+--+---+--+
-|126|lens.server.user.resolver.type|FIXED|Type of user config resolver. allowed values are FIXED, PROPERTYBASED, DATABASE, LDAP_BACKED_DATABASE, CUSTOM.|
+|127|lens.server.user.resolver.type|FIXED|Type of user config resolver. allowed values are FIXED, PROPERTYBASED, DATABASE, LDAP_BACKED_DATABASE, CUSTOM.|
 *--+--+---+--+
-|127|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.|
+|128|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.|
 *--+--+---+--+
-|128|lens.server.ws.featurenames|multipart,moxyjson,moxyjsonconfigresovler|These JAX-RS Feature(s) would be started in the specified order when lens-server starts up|
+|129|lens.server.ws.featurenames|multipart,moxyjson,moxyjsonconfigresovler|These JAX-RS Feature(s) would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|129|lens.server.ws.filternames|requestlogger,consistentState,serverMode|These JAX-RS filters would be started in the specified order when lens-server starts up|
+|130|lens.server.ws.filternames|requestlogger,consistentState,serverMode|These JAX-RS filters would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|130|lens.server.ws.listenernames|appevent|These listeners would be called in the specified order when lens-server starts up|
+|131|lens.server.ws.listenernames|appevent|These listeners would be called in the specified order when lens-server starts up|
 *--+--+---+--+
-|131|lens.server.ws.resourcenames|session,metastore,query,savedquery,quota,scheduler,index,log|These JAX-RS resources would be started in the specified order when lens-server starts up|
+|132|lens.server.ws.resourcenames|session,metastore,query,savedquery,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


[05/16] lens git commit: LENS-1384 : Update error message for data completeness check

Posted by pr...@apache.org.
LENS-1384 : Update error message for data completeness check


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

Branch: refs/heads/lens-1381
Commit: d46e78cff644b8436b80adc6f441799cfda4af27
Parents: 29568fa
Author: Narayan Periwal <na...@inmobi.com>
Authored: Fri Jan 27 10:11:33 2017 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Fri Jan 27 10:11:33 2017 +0530

----------------------------------------------------------------------
 .../apache/lens/cube/parse/CandidateTablePruneCause.java    | 9 +++++----
 .../java/org/apache/lens/cube/parse/TestCubeRewriter.java   | 6 ++----
 2 files changed, 7 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/d46e78cf/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTablePruneCause.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTablePruneCause.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTablePruneCause.java
index 2ad6e20..bd6e27c 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTablePruneCause.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTablePruneCause.java
@@ -148,15 +148,16 @@ public class CandidateTablePruneCause {
       }
     },
     // incomplete data in the fact
-    INCOMPLETE_PARTITION("Data is incomplete. Details : %s") {
+    INCOMPLETE_PARTITION("Data for the requested metrics is only partially complete. Partially complete metrics are:"
+            + " %s. Please try again later or rerun after removing incomplete metrics") {
       Object[] getFormatPlaceholders(Set<CandidateTablePruneCause> causes) {
-        Set<Map<String, Map<String, Float>>> incompletePartitions = Sets.newHashSet();
+        Set<String> incompleteMetrics = Sets.newHashSet();
         for (CandidateTablePruneCause cause : causes) {
           if (cause.getIncompletePartitions() != null) {
-            incompletePartitions.add(cause.getIncompletePartitions());
+            incompleteMetrics.addAll(cause.getIncompletePartitions().keySet());
           }
         }
-        return new String[]{incompletePartitions.toString()};
+        return new String[]{incompleteMetrics.toString()};
       }
     };
 

http://git-wip-us.apache.org/repos/asf/lens/blob/d46e78cf/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java
index c9e7c29..98b021b 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java
@@ -970,11 +970,9 @@ public class TestCubeRewriter extends TestQueryRewrite {
     assertEquals(e.getErrorCode(), LensCubeErrorCode.NO_CANDIDATE_FACT_AVAILABLE.getLensErrorInfo().getErrorCode());
     NoCandidateFactAvailableException ne = (NoCandidateFactAvailableException) e;
     PruneCauses.BriefAndDetailedError pruneCauses = ne.getJsonMessage();
-    /*Since the Flag FAIL_QUERY_ON_PARTIAL_DATA is set, and thhe queried fact has incomplete data, hence, we expect the
+    /*Since the Flag FAIL_QUERY_ON_PARTIAL_DATA is set, and the queried fact has incomplete data, hence, we expect the
     prune cause to be INCOMPLETE_PARTITION. The below check is to validate this.*/
-    assertEquals(pruneCauses.getBrief().substring(0, INCOMPLETE_PARTITION.errorFormat.length() - 3),
-            INCOMPLETE_PARTITION.errorFormat.substring(0,
-                    INCOMPLETE_PARTITION.errorFormat.length() - 3), pruneCauses.getBrief());
+    assertEquals(pruneCauses.getBrief(), String.format(INCOMPLETE_PARTITION.errorFormat, "[msr9]"));
   }
 
   @Test


[14/16] lens git commit: LENS-1398: Python client not raising exception in some cases

Posted by pr...@apache.org.
LENS-1398: Python client not raising exception in some cases


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

Branch: refs/heads/lens-1381
Commit: a3045fee12b3e750b6c931f5221a480d415efc76
Parents: bd344c7
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Tue Mar 21 16:09:05 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Tue Mar 21 16:09:05 2017 +0530

----------------------------------------------------------------------
 contrib/clients/python/lens/client/main.py     |  4 +-
 contrib/clients/python/lens/client/query.py    | 43 ++++++++++++++++-----
 contrib/clients/python/setup.py                |  2 +
 contrib/clients/python/test/test_lensclient.py | 13 +++++--
 tools/conf/server/lens-site.xml                |  6 +++
 5 files changed, 54 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/a3045fee/contrib/clients/python/lens/client/main.py
----------------------------------------------------------------------
diff --git a/contrib/clients/python/lens/client/main.py b/contrib/clients/python/lens/client/main.py
index bf5d81e..1e9d88f 100644
--- a/contrib/clients/python/lens/client/main.py
+++ b/contrib/clients/python/lens/client/main.py
@@ -21,7 +21,8 @@ from .log import LensLogClient
 from .session import LensSessionClient
 from .query import LensQueryClient
 from .utils import xml_file_to_conf
-
+import logging
+logger = logging.getLogger(__name__)
 
 class LensClient(object):
     def __init__(self, base_url=None, username="", password="", database=None, conf=None):
@@ -35,6 +36,7 @@ class LensClient(object):
         self.base_url = base_url or conf.get('lens.server.base.url', "http://0.0.0.0:9999/lensapi")
         if self.base_url[-1] != '/':
             self.base_url += "/"
+        logger.debug("Using conf %s", conf)
         username = username or conf.get('lens.client.user.name', "anonymous")
         database = database or conf.get('lens.client.dbname')
         self.session = LensSessionClient(self.base_url, username, password, database, conf)

http://git-wip-us.apache.org/repos/asf/lens/blob/a3045fee/contrib/clients/python/lens/client/query.py
----------------------------------------------------------------------
diff --git a/contrib/clients/python/lens/client/query.py b/contrib/clients/python/lens/client/query.py
index f82f0cb..df16cca 100644
--- a/contrib/clients/python/lens/client/query.py
+++ b/contrib/clients/python/lens/client/query.py
@@ -15,21 +15,26 @@
 # limitations under the License.
 #
 import codecs
+import csv
+import logging
 import time
 import zipfile
 
 import requests
-from six import string_types, BytesIO, StringIO, PY2, PY3
+from requests.exceptions import HTTPError
+from six import string_types, BytesIO, PY2, PY3
+
 from .models import WrappedJson
 from .utils import conf_to_xml
-import csv
 
+logger = logging.getLogger(__name__)
 long_type = int
 
 if PY3:
     from collections.abc import Iterable as Iterable
 elif PY2:
     from collections import Iterable as Iterable
+
     long_type = long
 
 
@@ -70,6 +75,7 @@ type_mappings = {'BOOLEAN': bool,
                  }
 default_mapping = lambda x: x
 
+
 class LensQueryResult(Iterable):
     def __init__(self, custom_mappings=None):
         if custom_mappings is None:
@@ -93,6 +99,7 @@ class LensInMemoryResult(LensQueryResult):
         for row in self.rows:
             yield list(self._mapping(value.type)(value.value) if value else None for value in row['values'])
 
+
 class LensPersistentResult(LensQueryResult):
     def __init__(self, header, response, encoding=None, is_header_present=True, delimiter=",",
                  custom_mappings=None):
@@ -107,6 +114,11 @@ class LensPersistentResult(LensQueryResult):
     def _parse_line(self, line):
         return list(self._mapping(self.header.columns[index].type)(line[index]) for index in range(len(line)))
 
+    def get_csv_reader(self, file):
+        if PY3:
+            file = codecs.iterdecode(file, 'utf-8')
+        return csv.reader(file, delimiter=self.delimiter)
+
     def __iter__(self):
         if self.is_zipped:
             byte_stream = BytesIO(self.response.content)
@@ -114,7 +126,7 @@ class LensPersistentResult(LensQueryResult):
                 for name in self.zipfile.namelist():
                     with self.zipfile.open(name) as single_file:
                         if name[-3:] == 'csv':
-                            reader = csv.reader(single_file, delimiter=self.delimiter)
+                            reader = self.get_csv_reader(single_file)
                         else:
                             reader = single_file
                         reader_iterator = iter(reader)
@@ -142,7 +154,8 @@ class LensQueryClient(object):
         self.launched_queries = []
         self.finished_queries = {}
         self.query_confs = {}
-        self.is_header_present_in_result = self._session['lens.query.output.write.header'].lower() in ['true', '1', 't', 'y', 'yes', 'yeah', 'yup']
+        self.is_header_present_in_result = self._session['lens.query.output.write.header'].lower() \
+                                           in ['true', '1', 't', 'y', 'yes', 'yeah', 'yup']
 
     def __call__(self, **filters):
         filters['sessionid'] = self._session._sessionid
@@ -183,6 +196,7 @@ class LensQueryClient(object):
         payload.append(('conf', conf_to_xml(conf)))
         resp = requests.post(self.base_url + "queries/", files=payload, headers={'accept': 'application/json'})
         query = self.sanitize_response(resp)
+        logger.info("Submitted query %s", query)
         if conf:
             self.query_confs[str(query)] = conf
         if fetch_result:
@@ -216,7 +230,8 @@ class LensQueryClient(object):
                 return LensPersistentResult(metadata, resp, is_header_present=is_header_present, *args, **kwargs)
             else:
                 response = requests.get(self.base_url + "queries/" + handle + "/resultset",
-                                    params={'sessionid': self._session._sessionid}, headers={'accept': 'application/json'})
+                                        params={'sessionid': self._session._sessionid},
+                                        headers={'accept': 'application/json'})
                 resp = self.sanitize_response(response)
                 # If it has in memory result, return inmemory result iterator
                 if resp._is_wrapper and resp._wrapped_key == u'inMemoryQueryResult':
@@ -228,13 +243,18 @@ class LensQueryClient(object):
             raise Exception("Result set not available")
 
     def sanitize_response(self, resp):
-        resp.raise_for_status()
         try:
             resp_json = resp.json(object_hook=WrappedJson)
+        except:
+            resp_json = resp.json()
+        if resp_json is not None:
             if 'lensAPIResult' in resp_json:
                 resp_json = resp_json.lens_a_p_i_result
                 if 'error' in resp_json:
-                    raise Exception(resp_json['error'])
+                    error = resp_json['error']
+                    if "stackTrace" in error:
+                        logger.error(error['stackTrace'])
+                    raise HTTPError(error, request=resp.request, response=resp)
                 if 'data' in resp_json:
                     data = resp_json.data
                     if len(data) == 2 and 'type' in data:
@@ -242,6 +262,9 @@ class LensQueryClient(object):
                         keys.remove('type')
                         return WrappedJson({data['type']: data[keys[0]]})
                     return data
-        except:
-            resp_json = resp.json()
-        return resp_json
+        if resp_json is not None:
+            return resp_json
+        else:
+            resp.raise_for_status()
+        logger.error(resp.text)
+        raise Exception("Unknown error with response", resp)

http://git-wip-us.apache.org/repos/asf/lens/blob/a3045fee/contrib/clients/python/setup.py
----------------------------------------------------------------------
diff --git a/contrib/clients/python/setup.py b/contrib/clients/python/setup.py
index 113205d..de59d32 100644
--- a/contrib/clients/python/setup.py
+++ b/contrib/clients/python/setup.py
@@ -45,6 +45,8 @@ class Tox(TestCommand):
     def run_tests(self):
         #import here, cause outside the eggs aren't loaded
         import tox
+        import os
+        del os.environ["PYTHONPATH"]
         errcode = tox.cmdline(self.test_args)
         sys.exit(errcode)
 

http://git-wip-us.apache.org/repos/asf/lens/blob/a3045fee/contrib/clients/python/test/test_lensclient.py
----------------------------------------------------------------------
diff --git a/contrib/clients/python/test/test_lensclient.py b/contrib/clients/python/test/test_lensclient.py
index e9b0bdb..b8d7c73 100644
--- a/contrib/clients/python/test/test_lensclient.py
+++ b/contrib/clients/python/test/test_lensclient.py
@@ -29,7 +29,9 @@ from lens.client.models import WrappedJson
 from requests.exceptions import HTTPError
 
 from lens.client import LensClient
-
+import logging
+logger = logging.getLogger(__name__)
+logging.basicConfig(level="DEBUG")
 
 def check_output(command):
     output = subprocess.check_output(command.split())
@@ -85,11 +87,14 @@ class TestLensClient(object):
         cls.base_path = glob.glob(joined)[0]
         with cwd(cls.base_path):
             with cwd('server'):
+                logger.info("Inside server directory")
                 server_start_output = check_output("bin/lens-ctl restart")
                 assert "Started lens server" in server_start_output
                 assert os.path.exists('logs/server.pid')
+                logger.info("started lens server")
                 time.sleep(1)
                 while not os.path.exists('logs/lensserver.log'):
+                    logger.info("waiting for lensserver.log to be created")
                     error = get_error()
                     if has_error(error):
                         # Assert again with complete error
@@ -97,9 +102,11 @@ class TestLensClient(object):
                     time.sleep(1)
                 error = get_error()
                 if has_error(error):
+                    logger.error(error)
                     assert False, error
-
+            logger.info("finished setting up server environment. Will setup client now")
             with cwd('client'):
+                logger.info("Inside client directory")
                 cls.candidate_query = select_query('examples/resources/cube-queries.sql')
                 with open('check_connection.sql', 'w') as f:
                     f.write('show databases')
@@ -160,7 +167,7 @@ class TestLensClient(object):
             handle = client.queries.submit(self.candidate_query, query_name="Candidate Query")
             finished_query = client.queries.wait_till_finish(handle)
             assert client.queries[handle] == finished_query
-            queries = client.queries(state='SUCCESSFUL', fromDate=finished_query.submission_time - 1,
+            queries = client.queries(state='successful,failed', fromDate=finished_query.submission_time - 1,
                                      toDate=finished_query.submission_time + 1)
             assert handle in queries
 

http://git-wip-us.apache.org/repos/asf/lens/blob/a3045fee/tools/conf/server/lens-site.xml
----------------------------------------------------------------------
diff --git a/tools/conf/server/lens-site.xml b/tools/conf/server/lens-site.xml
index b53ff25..c303198 100644
--- a/tools/conf/server/lens-site.xml
+++ b/tools/conf/server/lens-site.xml
@@ -36,4 +36,10 @@
   <name>datanucleus.schema.autoCreateTables</name>
   <value>true</value>
 </property>
+
+<property>
+  <name>hive.metastore.schema.verification</name>
+  <value>false</value>
+</property>
+
 </configuration>


[08/16] lens git commit: LENS-1380 : Revamp testcase division

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/client/ITSessionResourceTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/client/ITSessionResourceTests.java b/lens-regression/src/test/java/org/apache/lens/regression/client/ITSessionResourceTests.java
new file mode 100644
index 0000000..d39a2bd
--- /dev/null
+++ b/lens-regression/src/test/java/org/apache/lens/regression/client/ITSessionResourceTests.java
@@ -0,0 +1,401 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.lens.regression.client;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.GenericType;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import javax.xml.bind.JAXBException;
+
+import org.apache.lens.api.StringList;
+import org.apache.lens.api.query.LensQuery;
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+import org.apache.lens.api.session.UserSessionInfo;
+import org.apache.lens.regression.core.constants.MetastoreURL;
+import org.apache.lens.regression.core.constants.QueryInventory;
+import org.apache.lens.regression.core.constants.QueryURL;
+import org.apache.lens.regression.core.constants.SessionURL;
+import org.apache.lens.regression.core.helpers.ServiceManagerHelper;
+import org.apache.lens.regression.core.testHelper.BaseTestClass;
+import org.apache.lens.regression.core.type.FormBuilder;
+import org.apache.lens.regression.core.type.MapBuilder;
+import org.apache.lens.regression.util.AssertUtil;
+import org.apache.lens.regression.util.Util;
+import org.apache.lens.server.api.error.LensException;
+import org.apache.lens.server.api.util.LensUtil;
+
+import org.apache.log4j.Logger;
+
+import org.testng.Assert;
+import org.testng.annotations.*;
+
+import com.jcraft.jsch.JSchException;
+
+public class ITSessionResourceTests extends BaseTestClass {
+
+  WebTarget servLens;
+  private String sessionHandleString;
+
+  private final String hdfsJarPath = lens.getServerHdfsUrl() + "/tmp";
+  private final String localJarPath = new File("").getAbsolutePath() + "/lens-regression/target/testjars/";
+  private final String hiveUdfJar = "hiveudftest.jar";
+  private final String serverResourcePath = "/tmp/regression/resources";
+
+  private static String newParamsKey = "datanucleus.autoCreateSchema";
+  private static String newParamsValue = "false";
+  private static String createSleepFunction = "CREATE TEMPORARY FUNCTION sleep AS 'SampleUdf'";
+  private static Logger logger = Logger.getLogger(ITSessionResourceTests.class);
+
+
+  @BeforeClass(alwaysRun = true)
+  public void initialize() throws IOException, JSchException, JAXBException, LensException {
+    servLens = ServiceManagerHelper.init();
+  }
+
+  @BeforeMethod(alwaysRun = true)
+  public void setUp(Method method) throws Exception {
+    logger.info("Test Name: " + method.getName());
+    logger.info("Creating a new Session");
+    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+  }
+
+  @AfterMethod(alwaysRun = true)
+  public void closeSession() throws Exception {
+    logger.info("Closing Session");
+    sHelper.closeSession();
+  }
+
+  private boolean checkSessionParamMap(String sessionHandle) throws Exception {
+    MapBuilder query = new MapBuilder("sessionid", sessionHandle, "verbose", "true");
+    Response response = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query);
+    AssertUtil.assertSucceededResponse(response);
+    StringList strList = response.readEntity(new GenericType<StringList>(StringList.class));
+    HashMap<String, String> map = Util.stringListToMap(strList);
+    if (map == null) {
+      return false;
+    }
+    return true;
+  }
+
+
+  @Test
+  public void testSessionGet() throws Exception {
+    String newSessionHandle = sHelper.openSession("diff", "diff");
+    Assert.assertNotNull(newSessionHandle);
+  }
+
+
+  @Test
+  public void testSessionParamsGetVerbose() throws Exception {
+    Assert.assertTrue(checkSessionParamMap(sessionHandleString), "Returned Empty Params list");
+  }
+
+
+  @Test
+  public void testSessionParamsGetNPut() throws Exception {
+
+    Map<String, String> resource = LensUtil.getHashMap("sessionid", sessionHandleString, "key", newParamsKey,
+      "value", newParamsValue);
+    FormBuilder formData = new FormBuilder(resource);
+    Response response = lens.sendForm("put", SessionURL.SESSION_PARAMS_URL, formData);
+    AssertUtil.assertSucceededResult(response);
+
+    String value = sHelper.getSessionParam(newParamsKey);
+    Assert.assertEquals(value, newParamsValue, "From Session Params Put");
+
+  }
+
+  //Negative Test Case
+  @Test
+  public void testSessionGetUndefinedParams() throws Exception {
+    String undefinedParamsKey = "test123";
+    MapBuilder query = new MapBuilder("sessionid", sessionHandleString, "key", undefinedParamsKey);
+    Response response = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query);
+    Assert.assertEquals(response.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
+  }
+
+/*
+ * Testing if Session is restored after server restart
+ */
+
+  @Test
+  public void testSessionRestore() throws Exception {
+
+    Map<String, String> resource = LensUtil.getHashMap("sessionid", sessionHandleString,
+      "key", newParamsKey, "value", newParamsValue);
+    FormBuilder formData = new FormBuilder(resource);
+
+    Response response = lens.sendForm("put", SessionURL.SESSION_PARAMS_URL, formData);
+    AssertUtil.assertSucceededResult(response);
+
+    lens.restart();
+
+    MapBuilder query = new MapBuilder("sessionid", sessionHandleString, "key", newParamsKey);
+    response = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query);
+    AssertUtil.assertSucceededResponse(response);
+    StringList strList = response.readEntity(new GenericType<StringList>(StringList.class));
+    HashMap<String, String> map = Util.stringListToMap(strList);
+
+    Assert.assertEquals(map.get(newParamsKey), newParamsValue, "From Session Params Put");
+    Assert.assertEquals(map.size(), 1, "Params List contains more than one param");
+  }
+
+  @Test(enabled = true)
+  public void testSessionHDFSResourcePutNDelete() throws Exception {
+
+    String path = hdfsJarPath + "/" + hiveUdfJar;
+    sHelper.addResourcesJar(path);
+
+    QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(createSleepFunction).getData();
+    LensQuery lensQuery = qHelper.waitForCompletion(queryHandle);
+    Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+
+    sHelper.removeResourcesJar(path);
+
+    queryHandle = (QueryHandle) qHelper.executeQuery(createSleepFunction).getData();
+    lensQuery = qHelper.waitForCompletion(queryHandle);
+    // TODO : Works only when there is single instance for each driver
+//    Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.FAILED);
+
+  }
+
+
+  @Test
+  public void testSessionLocalResourcePutNDelete() throws Exception {
+
+    String path = serverResourcePath + "/" + hiveUdfJar;
+    sHelper.addResourcesJar(path);
+
+    QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(createSleepFunction).getData();
+    LensQuery lensQuery = qHelper.waitForCompletion(queryHandle);
+    Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+
+    sHelper.removeResourcesJar(path);
+
+    queryHandle = (QueryHandle) qHelper.executeQuery(createSleepFunction).getData();
+    lensQuery = qHelper.waitForCompletion(queryHandle);
+//  TODO : Works only when there is single instance for each driver
+//  Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.FAILED);
+
+  }
+
+  @Test
+  public void testListResources() throws Exception {
+
+    String path = serverResourcePath + "/" + hiveUdfJar;
+    sHelper.addResourcesJar(path);
+
+    MapBuilder query = new MapBuilder("sessionid", sessionHandleString);
+    Response response = lens.sendQuery("get", SessionURL.SESSION_LIST_RESOURCE_URL, query);
+    AssertUtil.assertSucceededResponse(response);
+    StringList responseString = response.readEntity(StringList.class);
+    List<String> jars = responseString.getElements();
+    for (String t : jars) {
+      Assert.assertTrue(t.contains(hiveUdfJar));
+    }
+  }
+
+
+  @Test
+  public void testSessionGone() throws Exception {
+
+    String newSession = sHelper.openSession("test", "test");
+    sHelper.closeSession(newSession);
+
+    MapBuilder query = new MapBuilder("sessionid", newSession);
+
+    // Get Session resources with closed session
+    Response response = lens.sendQuery("get", SessionURL.SESSION_LIST_RESOURCE_URL, query);
+    AssertUtil.assertGone(response);
+
+    // Get Session params with closd session
+    response = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query);
+    AssertUtil.assertGone(response);
+
+    //Setting DB with closed session Handle
+    response = lens.exec("post", MetastoreURL.METASTORE_DATABASES_URL, servLens,
+        null, query, MediaType.APPLICATION_XML_TYPE, null, lens.getCurrentDB());
+    AssertUtil.assertGone(response);
+
+    FormBuilder formData = new FormBuilder();
+    formData.add("sessionid", newSession);
+    formData.add("query", QueryInventory.QUERY);
+    formData.add("conf", "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?><conf />");
+
+    //Explain Query with closed session Handle
+    formData.add("operation", "EXPLAIN");
+    response = lens.exec("post", QueryURL.QUERY_URL, servLens, null, null,
+        MediaType.MULTIPART_FORM_DATA_TYPE, MediaType.APPLICATION_XML, formData.getForm());
+    AssertUtil.assertGone(response);
+
+    //Execute Query with closed session Handle
+    formData.add("operation", "EXECUTE");
+    response = lens.exec("post", QueryURL.QUERY_URL, servLens, null, null,
+        MediaType.MULTIPART_FORM_DATA_TYPE, MediaType.APPLICATION_XML, formData.getForm());
+    AssertUtil.assertGone(response);
+
+  }
+
+  @Test
+  public void testOpenSessionWithDB() throws Exception {
+
+    String newDb = "opensessionwithdb";
+    mHelper.createDatabase(newDb);
+    String newSession = sHelper.openSession("test", "test", newDb);
+    String curDB = mHelper.getCurrentDatabase(newSession);
+    Assert.assertEquals(curDB, newDb, "Could not open session with passed db");
+    sHelper.closeSession(newSession);
+    mHelper.dropDatabase(newDb);
+  }
+
+  @Test
+  public void testOpenSessionDefault() throws Exception {
+
+    String newSession = sHelper.openSession("test", "test");
+    String curDB = mHelper.getCurrentDatabase(newSession);
+    Assert.assertEquals(curDB, "default", "Could not open session with passed db");
+    sHelper.closeSession(newSession);
+  }
+
+
+  @Test
+  public void testOpenSessionDBDoesnotExist() throws Exception {
+
+    Response response = sHelper.openSessionReturnResponse("test", "test", "dbdoesnotexist", null);
+    AssertUtil.assertNotFound(response);
+  }
+
+  @Test
+  public void testSessionDBChange() throws Exception {
+
+    String newDb = "opensessionwithdb";
+    String newDb1 = "opensessionwithdb1";
+    mHelper.createDatabase(newDb);
+    mHelper.createDatabase(newDb1);
+
+    String newSession = sHelper.openSession("test", "test", newDb);
+    String curDB = mHelper.getCurrentDatabase(newSession);
+    Assert.assertEquals(curDB, newDb, "Could not open session with passed db");
+
+    mHelper.setCurrentDatabase(newSession, newDb1);
+    curDB = mHelper.getCurrentDatabase(newSession);
+    Assert.assertEquals(curDB, newDb1, "Could not open session with passed db");
+
+    sHelper.closeSession(newSession);
+    mHelper.dropDatabase(newDb);
+    mHelper.dropDatabase(newDb1);
+  }
+
+  //Fails as closeSession cannot take json as input,. (No API can take json as input)
+  @Test(enabled = false)
+  public void testGetSessionJson() throws Exception {
+
+    String newSessionHandle = sHelper.openSession("diff", "diff", null, MediaType.APPLICATION_JSON);
+    Assert.assertNotNull(newSessionHandle);
+    Assert.assertFalse(newSessionHandle.isEmpty());
+    sHelper.closeSession(newSessionHandle, MediaType.APPLICATION_JSON);
+  }
+
+  @Test(enabled = true)
+  public void assertSucceededResponse() throws Exception {
+    String session = sHelper.openSession("diff", "diff", null, MediaType.APPLICATION_XML);
+    Assert.assertNotNull(session);
+    Assert.assertFalse(session.isEmpty());
+
+    MapBuilder query = new MapBuilder("sessionid", session);
+    Response response = lens.exec("delete", SessionURL.SESSION_BASE_URL, servLens, null, query, null,
+        MediaType.APPLICATION_JSON, null);
+    AssertUtil.assertSucceededResponse(response);
+  }
+
+
+  @Test(enabled = true)
+  public void listSessionTest() throws Exception {
+
+    int origSize = sHelper.getSessionList().size();
+    for(int i=1; i<4; i++) {
+      sHelper.openSession("u" + i, "p" + i);
+    }
+    List<UserSessionInfo> sessionList = sHelper.getSessionList();
+    Assert.assertEquals(sessionList.size(), origSize+3);
+  }
+
+  //TODO : enable when session handle returned is entire xml instead of just public id
+  @Test(enabled = false)
+  public void listSessionUserSessionInfoVerification() throws Exception {
+
+    List<UserSessionInfo> sessionList = sHelper.getSessionList();
+    for(UserSessionInfo u : sessionList){
+      System.out.println(u.toString() + "\n");
+      sHelper.closeSession(u.getHandle());
+    }
+
+    String session1 = sHelper.openSession("u1", "p1", lens.getCurrentDB());
+    String session2 = sHelper.openSession("u2", "p2", lens.getCurrentDB());
+
+    QueryHandle qh1 = (QueryHandle) qHelper.executeQuery(QueryInventory.JDBC_DIM_QUERY, null, session1).getData();
+    QueryHandle qh2 = (QueryHandle) qHelper.executeQuery(QueryInventory.JDBC_DIM_QUERY, null, session2).getData();
+
+    sessionList = sHelper.getSessionList();
+    Assert.assertEquals(sessionList.size(), 2);
+
+    UserSessionInfo s1 = sessionList.get(0);
+    Assert.assertEquals(s1.getUserName(), "u1");
+    List<QueryHandle> queryHandleList = s1.getActiveQueries();
+    Assert.assertEquals(queryHandleList.size(), 2);
+    Assert.assertTrue(queryHandleList.contains(qh1));
+    Assert.assertTrue(queryHandleList.contains(qh2));
+
+    for(UserSessionInfo u : sessionList){
+      System.out.println(u.toString() + "\n");
+      sHelper.closeSession(u.getHandle());
+    }
+  }
+
+  //LENS-1199
+  @Test(enabled = true)
+  public void multipleCloseSessionForActiveQueries() throws Exception {
+    String session = sHelper.openSession("diff", "diff", lens.getCurrentDB());
+    QueryHandle qh = (QueryHandle) qHelper.executeQuery(QueryInventory.getSleepQuery("3"), null, session).getData();
+    sHelper.closeSession(session);
+    sHelper.closeSession(session);
+  }
+
+  //LENS-1199
+  @Test(enabled = true)
+  public void multipleCloseSession() throws Exception {
+    String session = sHelper.openSession("diff", "diff", lens.getCurrentDB());
+    sHelper.closeSession(session);
+    MapBuilder query = new MapBuilder("sessionid", session);
+    Response response = lens.exec("delete", SessionURL.SESSION_BASE_URL, servLens, null, query);
+    AssertUtil.assertGone(response);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/client/KillQueryTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/client/KillQueryTests.java b/lens-regression/src/test/java/org/apache/lens/regression/client/KillQueryTests.java
deleted file mode 100644
index a39b2fe..0000000
--- a/lens-regression/src/test/java/org/apache/lens/regression/client/KillQueryTests.java
+++ /dev/null
@@ -1,362 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.lens.regression.client;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.reflect.Method;
-
-import javax.ws.rs.client.WebTarget;
-
-import javax.xml.bind.JAXBException;
-
-import org.apache.lens.api.query.*;
-import org.apache.lens.regression.core.constants.QueryInventory;
-import org.apache.lens.regression.core.helpers.*;
-import org.apache.lens.regression.core.testHelper.BaseTestClass;
-import org.apache.lens.server.api.error.LensException;
-
-import org.apache.log4j.Logger;
-
-import org.testng.Assert;
-import org.testng.annotations.*;
-
-
-public class KillQueryTests extends BaseTestClass {
-
-  WebTarget servLens;
-  private String sessionHandleString;
-
-  private final String hdfsJarPath = lens.getServerHdfsUrl() + "/tmp";
-  private final String localJarPath = new File("").getAbsolutePath() + "/lens-regression/target/testjars/";
-  private final String hiveUdfJar = "hiveudftest.jar";
-  private final String serverResourcePath = "/tmp/regression/resources";
-  String sleepQuery = QueryInventory.getSleepQuery("5");
-
-  private static Logger logger = Logger.getLogger(KillQueryTests.class);
-
-  @BeforeClass(alwaysRun = true)
-  public void initialize() throws IOException, JAXBException, LensException, IllegalAccessException,
-      InstantiationException {
-    servLens = ServiceManagerHelper.init();
-    logger.info("Creating a new Session");
-    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
-
-    //TODO : Enable when udf registration per driver is fixed
-/*  HadoopUtil.uploadJars(localJarPath + "/" + hiveUdfJar, hdfsJarPath);
-    logger.info("Adding jar for making query to run for longer period of time");
-    sHelper.addResourcesJar(hdfsJarPath + "/" + hiveUdfJar);
-    QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(QueryInventory.SLEEP_FUNCTION).getData();*/
-  }
-
-  @BeforeMethod(alwaysRun = true)
-  public void setUp(Method method) throws Exception {
-    logger.info("Test Name: " + method.getName());
-  }
-
-
-  @AfterClass(alwaysRun = true)
-  public void closeSession() throws Exception {
-    logger.info("Closing Session");
-    sHelper.closeSession();
-  }
-
-
-  @Test(enabled = true)
-  public void killQueryByHandle() throws Exception {
-
-    QueryHandle qH = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
-    logger.info("QUERY HANDLE : " + qH);
-
-    QueryStatus queryStatus = qHelper.waitForQueryToRun(qH);
-    Assert.assertEquals(queryStatus.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-
-    qHelper.killQueryByQueryHandle(qH);
-
-    queryStatus = qHelper.getQueryStatus(qH);
-    Assert.assertEquals(queryStatus.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-  }
-
-
-  @Test(enabled = true)
-  public void killQueryByUser() throws Exception {
-
-    String diffUser = "diff";
-    String diffPass = "diff";
-
-    String newSessionHandleSring = sHelper.openSession(diffUser, diffPass, lens.getCurrentDB());
-
-    logger.info("Adding jar for making query to run for longer period of time");
-    sHelper.addResourcesJar(hdfsJarPath + "/" + hiveUdfJar, newSessionHandleSring);
-
-    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
-    logger.info("1st QUERY HANDLE : " + queryHandle1);
-
-    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
-    logger.info("2nd QUERY HANDLE : " + queryHandle2);
-
-    QueryHandle queryHandle3 = (QueryHandle) qHelper.executeQuery(sleepQuery, null,
-        newSessionHandleSring).getData();
-    logger.info("3rd QUERY HANDLE : " + queryHandle3);
-
-    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
-    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
-    QueryStatus queryStatus3 = qHelper.waitForQueryToRun(queryHandle3, newSessionHandleSring);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-
-    Thread.sleep(1000);
-
-    qHelper.killQuery(null, null, lens.getUserName());
-    Thread.sleep(2000);
-    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
-    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
-    queryStatus3 = qHelper.getQueryStatus(queryHandle3);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-
-    qHelper.killQuery(null, null, diffUser);
-
-    queryStatus3 = qHelper.getQueryStatus(queryHandle3);
-    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-
-  }
-
-  @Test(enabled = true)
-  public void killQueryOfAllUser() throws Exception {
-
-    String diffUser = "diff";
-    String diffPass = "diff";
-    String newSessionHandleSring = sHelper.openSession(diffUser, diffPass, lens.getCurrentDB());
-    sHelper.addResourcesJar(hdfsJarPath + "/" + hiveUdfJar, newSessionHandleSring);
-
-    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
-    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
-    QueryHandle queryHandle3 = (QueryHandle) qHelper.executeQuery(sleepQuery, null,
-        newSessionHandleSring).getData();
-
-    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
-    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
-    QueryStatus queryStatus3 = qHelper.waitForQueryToRun(queryHandle3, newSessionHandleSring);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-
-    qHelper.killQuery(null, null, "all");
-
-    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
-    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
-    queryStatus3 = qHelper.getQueryStatus(queryHandle3);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-  }
-
-
-  @Test(enabled = true)
-  public void killAllQueryOfUser() throws Exception {
-
-    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
-    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
-
-    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
-    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-
-    qHelper.killQuery();
-
-    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
-    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-  }
-
-
-  @Test(enabled = true)
-  public void killQueryByState() throws Exception {
-
-    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
-    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
-
-    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
-    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-
-    //kill Running queries
-    qHelper.killQuery(null, "RUNNING");
-
-    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
-    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-
-    //kill Canceled query
-    qHelper.killQuery(null, "CANCELED");
-
-    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
-    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
-
-    //kill successful query
-    QueryHandle queryHandle3 = (QueryHandle) qHelper.executeQuery(QueryInventory.QUERY).getData();
-    LensQuery lensQuery = qHelper.waitForCompletion(queryHandle3);
-    Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-    qHelper.killQuery(null, "SUCCESSFUL");
-    QueryStatus queryStatus3 = qHelper.getQueryStatus(queryHandle3);
-    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.SUCCESSFUL);
-  }
-
-  //TODO: enable when the bug is fixed.
-
-   /* Currently doing kill query by queryName "query" will kill all the query with queryName as "*query*"
-    * Raised a JIRA for same
-    * When its Fixed Revisit this function */
-
-  @Test(enabled = false)
-  public void killQueryByQueryName() throws Exception {
-
-    String queryName1 = "queryNameFirst";
-    String queryName2 = "queryNameSecond";
-    String queryName3 = "Name";
-
-    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery, queryName1).getData();
-    logger.info("1st QUERY HANDLE : " + queryHandle1);
-
-    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery, queryName2).getData();
-    logger.info("2nd QUERY HANDLE : " + queryHandle2);
-
-    QueryHandle queryHandle3 = (QueryHandle) qHelper.executeQuery(sleepQuery, queryName3).getData();
-    logger.info("3rd QUERY HANDLE : " + queryHandle3);
-
-    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
-    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
-    QueryStatus queryStatus3 = qHelper.waitForQueryToRun(queryHandle3);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-
-    qHelper.killQuery(queryName3);
-
-    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
-    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
-    queryStatus3 = qHelper.getQueryStatus(queryHandle3);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Cancelled");
-
-    qHelper.killQuery(queryName1);
-
-    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
-    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not CANCELED");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-
-    qHelper.killQuery(queryName2);
-    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
-
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not CANCELED");
-  }
-
-  @Test(enabled = true)
-  public void killQueryByTimeRange() throws Exception {
-
-    String startTime1 = String.valueOf(System.currentTimeMillis());
-    logger.info("Start Time of 1st Query : " + startTime1);
-    Thread.sleep(1000);
-    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
-    Thread.sleep(1000);
-    String endTime1 = String.valueOf(System.currentTimeMillis());
-    logger.info("End Time of 1st Query : " + endTime1);
-
-    Thread.sleep(1000);
-
-    String startTime2 = String.valueOf(System.currentTimeMillis());
-    logger.info("Start Time of 2nd Query : " + startTime2);
-    Thread.sleep(1000);
-    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
-    Thread.sleep(1000);
-    String endTime2 = String.valueOf(System.currentTimeMillis());
-    logger.info("End Time of 2nd Query : " + endTime2);
-
-    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
-    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-
-    qHelper.killQuery(null, null, null, sessionHandleString, startTime1, endTime1);
-
-    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
-    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
-
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not CANCELED");
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not CANCELED");
-
-    qHelper.killQuery(null, null, null, sessionHandleString, startTime2, endTime2);
-
-    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
-    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not CANCELED");
-
-  }
-
-
-  @Test(enabled = true)
-  public void killQueryByAllFilter() throws Exception {
-
-    String queryName1 = "TestKill";
-
-    String startTime1 = String.valueOf(System.currentTimeMillis());
-    logger.info("Start Time of 1st Query : " + startTime1);
-    Thread.sleep(1000);
-
-    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery, queryName1).getData();
-    Thread.sleep(1000);
-
-    String endTime1 = String.valueOf(System.currentTimeMillis());
-    logger.info("End Time of 1st Query : " + endTime1);
-
-    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
-
-    qHelper.killQuery(queryName1, "RUNNING", lens.getUserName(), sessionHandleString, startTime1, endTime1);
-
-    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
-    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not CANCELED");
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/client/SessionResourceTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/client/SessionResourceTests.java b/lens-regression/src/test/java/org/apache/lens/regression/client/SessionResourceTests.java
deleted file mode 100644
index f6a10e8..0000000
--- a/lens-regression/src/test/java/org/apache/lens/regression/client/SessionResourceTests.java
+++ /dev/null
@@ -1,403 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.lens.regression.client;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import javax.ws.rs.client.WebTarget;
-import javax.ws.rs.core.GenericType;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-
-import javax.xml.bind.JAXBException;
-
-import org.apache.lens.api.StringList;
-import org.apache.lens.api.query.LensQuery;
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryStatus;
-import org.apache.lens.api.session.UserSessionInfo;
-import org.apache.lens.regression.core.constants.MetastoreURL;
-import org.apache.lens.regression.core.constants.QueryInventory;
-import org.apache.lens.regression.core.constants.QueryURL;
-import org.apache.lens.regression.core.constants.SessionURL;
-import org.apache.lens.regression.core.helpers.ServiceManagerHelper;
-import org.apache.lens.regression.core.testHelper.BaseTestClass;
-import org.apache.lens.regression.core.type.FormBuilder;
-import org.apache.lens.regression.core.type.MapBuilder;
-import org.apache.lens.regression.util.AssertUtil;
-import org.apache.lens.regression.util.Util;
-import org.apache.lens.server.api.error.LensException;
-import org.apache.lens.server.api.util.LensUtil;
-
-import org.apache.log4j.Logger;
-
-import org.testng.Assert;
-import org.testng.annotations.*;
-
-import com.jcraft.jsch.JSchException;
-
-public class SessionResourceTests extends BaseTestClass {
-
-  WebTarget servLens;
-  private String sessionHandleString;
-
-  private final String hdfsJarPath = lens.getServerHdfsUrl() + "/tmp";
-  private final String localJarPath = new File("").getAbsolutePath() + "/lens-regression/target/testjars/";
-  private final String hiveUdfJar = "hiveudftest.jar";
-  private final String serverResourcePath = "/tmp/regression/resources";
-
-  private static String newParamsKey = "datanucleus.autoCreateSchema";
-  private static String newParamsValue = "false";
-  private static String createSleepFunction = "CREATE TEMPORARY FUNCTION sleep AS 'SampleUdf'";
-  private static Logger logger = Logger.getLogger(SessionResourceTests.class);
-
-
-  @BeforeClass(alwaysRun = true)
-  public void initialize() throws IOException, JSchException, JAXBException, LensException {
-    servLens = ServiceManagerHelper.init();
-  }
-
-  @BeforeMethod(alwaysRun = true)
-  public void setUp(Method method) throws Exception {
-    logger.info("Test Name: " + method.getName());
-    logger.info("Creating a new Session");
-    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
-  }
-
-  @AfterMethod(alwaysRun = true)
-  public void closeSession() throws Exception {
-    logger.info("Closing Session");
-    sHelper.closeSession();
-  }
-
-  private boolean checkSessionParamMap(String sessionHandle) throws Exception {
-    MapBuilder query = new MapBuilder("sessionid", sessionHandle, "verbose", "true");
-    Response response = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query);
-    AssertUtil.assertSucceededResponse(response);
-    StringList strList = response.readEntity(new GenericType<StringList>(StringList.class));
-    HashMap<String, String> map = Util.stringListToMap(strList);
-    if (map == null) {
-      return false;
-    }
-    return true;
-  }
-
-
-  @Test
-  public void testSessionGet() throws Exception {
-    String newSessionHandle = sHelper.openSession("diff", "diff");
-    Assert.assertNotNull(newSessionHandle);
-  }
-
-
-  @Test
-  public void testSessionParamsGetVerbose() throws Exception {
-    Assert.assertTrue(checkSessionParamMap(sessionHandleString), "Returned Empty Params list");
-  }
-
-
-  @Test
-  public void testSessionParamsGetNPut() throws Exception {
-
-    Map<String, String> resource = LensUtil.getHashMap("sessionid", sessionHandleString, "key", newParamsKey,
-      "value", newParamsValue);
-    FormBuilder formData = new FormBuilder(resource);
-    Response response = lens.sendForm("put", SessionURL.SESSION_PARAMS_URL, formData);
-    AssertUtil.assertSucceededResult(response);
-
-    String value = sHelper.getSessionParam(newParamsKey);
-    Assert.assertEquals(value, newParamsValue, "From Session Params Put");
-
-  }
-
-  //Negative Test Case
-  @Test
-  public void testSessionGetUndefinedParams() throws Exception {
-    String undefinedParamsKey = "test123";
-    MapBuilder query = new MapBuilder("sessionid", sessionHandleString, "key", undefinedParamsKey);
-    Response response = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query);
-    Assert.assertEquals(response.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
-  }
-
-/*
- * Testing if Session is restored after server restart
- */
-
-  @Test
-  public void testSessionRestore() throws Exception {
-
-    Map<String, String> resource = LensUtil.getHashMap("sessionid", sessionHandleString,
-      "key", newParamsKey, "value", newParamsValue);
-    FormBuilder formData = new FormBuilder(resource);
-
-    Response response = lens.sendForm("put", SessionURL.SESSION_PARAMS_URL, formData);
-    AssertUtil.assertSucceededResult(response);
-
-    lens.restart();
-
-    MapBuilder query = new MapBuilder("sessionid", sessionHandleString, "key", newParamsKey);
-    response = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query);
-    AssertUtil.assertSucceededResponse(response);
-    StringList strList = response.readEntity(new GenericType<StringList>(StringList.class));
-    HashMap<String, String> map = Util.stringListToMap(strList);
-
-    Assert.assertEquals(map.get(newParamsKey), newParamsValue, "From Session Params Put");
-    Assert.assertEquals(map.size(), 1, "Params List contains more than one param");
-  }
-
-  @Test(enabled = true)
-  public void testSessionHDFSResourcePutNDelete() throws Exception {
-
-    String path = hdfsJarPath + "/" + hiveUdfJar;
-    sHelper.addResourcesJar(path);
-
-    QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(createSleepFunction).getData();
-    LensQuery lensQuery = qHelper.waitForCompletion(queryHandle);
-    Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-
-    sHelper.removeResourcesJar(path);
-
-    queryHandle = (QueryHandle) qHelper.executeQuery(createSleepFunction).getData();
-    lensQuery = qHelper.waitForCompletion(queryHandle);
-    // TODO : Works only when there is single instance for each driver
-//    Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.FAILED);
-
-  }
-
-
-  @Test
-  public void testSessionLocalResourcePutNDelete() throws Exception {
-
-    String path = serverResourcePath + "/" + hiveUdfJar;
-    sHelper.addResourcesJar(path);
-
-    QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(createSleepFunction).getData();
-    LensQuery lensQuery = qHelper.waitForCompletion(queryHandle);
-    Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-
-    sHelper.removeResourcesJar(path);
-
-    queryHandle = (QueryHandle) qHelper.executeQuery(createSleepFunction).getData();
-    lensQuery = qHelper.waitForCompletion(queryHandle);
-//  TODO : Works only when there is single instance for each driver
-//  Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.FAILED);
-
-  }
-
-  @Test
-  public void testListResources() throws Exception {
-
-    String path = serverResourcePath + "/" + hiveUdfJar;
-    sHelper.addResourcesJar(path);
-
-    MapBuilder query = new MapBuilder("sessionid", sessionHandleString);
-    Response response = lens.sendQuery("get", SessionURL.SESSION_LIST_RESOURCE_URL, query);
-    AssertUtil.assertSucceededResponse(response);
-    StringList responseString = response.readEntity(StringList.class);
-    List<String> jars = responseString.getElements();
-    for (String t : jars) {
-      Assert.assertTrue(t.contains(hiveUdfJar));
-    }
-  }
-
-
-  @Test
-  public void testSessionGone() throws Exception {
-
-    String newSession = sHelper.openSession("test", "test");
-    sHelper.closeSession(newSession);
-
-    MapBuilder query = new MapBuilder("sessionid", newSession);
-
-    // Get Session resources with closed session
-    Response response = lens.sendQuery("get", SessionURL.SESSION_LIST_RESOURCE_URL, query);
-    AssertUtil.assertGone(response);
-
-    // Get Session params with closd session
-    response = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query);
-    AssertUtil.assertGone(response);
-
-    //Setting DB with closed session Handle
-    response = lens.exec("post", MetastoreURL.METASTORE_DATABASES_URL, servLens,
-        null, query, MediaType.APPLICATION_XML_TYPE, null, lens.getCurrentDB());
-    AssertUtil.assertGone(response);
-
-    FormBuilder formData = new FormBuilder();
-    formData.add("sessionid", newSession);
-    formData.add("query", QueryInventory.QUERY);
-    formData.add("conf", "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?><conf />");
-
-    //Explain Query with closed session Handle
-    formData.add("operation", "EXPLAIN");
-    response = lens.exec("post", QueryURL.QUERY_URL, servLens, null, null,
-        MediaType.MULTIPART_FORM_DATA_TYPE, MediaType.APPLICATION_XML, formData.getForm());
-    AssertUtil.assertGone(response);
-
-    //Execute Query with closed session Handle
-    formData.add("operation", "EXECUTE");
-    response = lens.exec("post", QueryURL.QUERY_URL, servLens, null, null,
-        MediaType.MULTIPART_FORM_DATA_TYPE, MediaType.APPLICATION_XML, formData.getForm());
-    AssertUtil.assertGone(response);
-
-  }
-
-  @Test
-  public void testOpenSessionWithDB() throws Exception {
-
-    String newDb = "opensessionwithdb";
-    mHelper.createDatabase(newDb);
-    String newSession = sHelper.openSession("test", "test", newDb);
-    String curDB = mHelper.getCurrentDatabase(newSession);
-    Assert.assertEquals(curDB, newDb, "Could not open session with passed db");
-    sHelper.closeSession(newSession);
-    //TODO : Enable when drop table is fixed
-//    mHelper.dropDatabase(newDb);
-  }
-
-  @Test
-  public void testOpenSessionDefault() throws Exception {
-
-    String newSession = sHelper.openSession("test", "test");
-    String curDB = mHelper.getCurrentDatabase(newSession);
-    Assert.assertEquals(curDB, "default", "Could not open session with passed db");
-    sHelper.closeSession(newSession);
-  }
-
-
-  @Test
-  public void testOpenSessionDBDoesnotExist() throws Exception {
-
-    Response response = sHelper.openSessionReturnResponse("test", "test", "dbdoesnotexist", null);
-    AssertUtil.assertNotFound(response);
-  }
-
-  @Test
-  public void testSessionDBChange() throws Exception {
-
-    String newDb = "opensessionwithdb";
-    String newDb1 = "opensessionwithdb1";
-    mHelper.createDatabase(newDb);
-    mHelper.createDatabase(newDb1);
-
-    String newSession = sHelper.openSession("test", "test", newDb);
-    String curDB = mHelper.getCurrentDatabase(newSession);
-    Assert.assertEquals(curDB, newDb, "Could not open session with passed db");
-
-    mHelper.setCurrentDatabase(newSession, newDb1);
-    curDB = mHelper.getCurrentDatabase(newSession);
-    Assert.assertEquals(curDB, newDb1, "Could not open session with passed db");
-
-    sHelper.closeSession(newSession);
-    //TODO : Enable when drop table issue is fixed
-//    mHelper.dropDatabase(newDb);
-//    mHelper.dropDatabase(newDb1);
-  }
-
-  //Fails as closeSession cannot take json as input,. (No API can take json as input)
-  @Test(enabled = false)
-  public void testGetSessionJson() throws Exception {
-
-    String newSessionHandle = sHelper.openSession("diff", "diff", null, MediaType.APPLICATION_JSON);
-    Assert.assertNotNull(newSessionHandle);
-    Assert.assertFalse(newSessionHandle.isEmpty());
-    sHelper.closeSession(newSessionHandle, MediaType.APPLICATION_JSON);
-  }
-
-  @Test(enabled = true)
-  public void assertSucceededResponse() throws Exception {
-    String session = sHelper.openSession("diff", "diff", null, MediaType.APPLICATION_XML);
-    Assert.assertNotNull(session);
-    Assert.assertFalse(session.isEmpty());
-
-    MapBuilder query = new MapBuilder("sessionid", session);
-    Response response = lens.exec("delete", SessionURL.SESSION_BASE_URL, servLens, null, query, null,
-        MediaType.APPLICATION_JSON, null);
-    AssertUtil.assertSucceededResponse(response);
-  }
-
-
-  @Test(enabled = true)
-  public void listSessionTest() throws Exception {
-
-    int origSize = sHelper.getSessionList().size();
-    for(int i=1; i<4; i++) {
-      sHelper.openSession("u" + i, "p" + i);
-    }
-    List<UserSessionInfo> sessionList = sHelper.getSessionList();
-    Assert.assertEquals(sessionList.size(), origSize+3);
-  }
-
-  //TODO : enable when session handle returned is entire xml instead of just public id
-  @Test(enabled = false)
-  public void listSessionUserSessionInfoVerification() throws Exception {
-
-    List<UserSessionInfo> sessionList = sHelper.getSessionList();
-    for(UserSessionInfo u : sessionList){
-      System.out.println(u.toString() + "\n");
-      sHelper.closeSession(u.getHandle());
-    }
-
-    String session1 = sHelper.openSession("u1", "p1", lens.getCurrentDB());
-    String session2 = sHelper.openSession("u2", "p2", lens.getCurrentDB());
-
-    QueryHandle qh1 = (QueryHandle) qHelper.executeQuery(QueryInventory.JDBC_DIM_QUERY, null, session1).getData();
-    QueryHandle qh2 = (QueryHandle) qHelper.executeQuery(QueryInventory.JDBC_DIM_QUERY, null, session2).getData();
-
-    sessionList = sHelper.getSessionList();
-    Assert.assertEquals(sessionList.size(), 2);
-
-    UserSessionInfo s1 = sessionList.get(0);
-    Assert.assertEquals(s1.getUserName(), "u1");
-    List<QueryHandle> queryHandleList = s1.getActiveQueries();
-    Assert.assertEquals(queryHandleList.size(), 2);
-    Assert.assertTrue(queryHandleList.contains(qh1));
-    Assert.assertTrue(queryHandleList.contains(qh2));
-
-    for(UserSessionInfo u : sessionList){
-      System.out.println(u.toString() + "\n");
-      sHelper.closeSession(u.getHandle());
-    }
-  }
-
-  //LENS-1199
-  @Test(enabled = true)
-  public void multipleCloseSessionForActiveQueries() throws Exception {
-    String session = sHelper.openSession("diff", "diff", lens.getCurrentDB());
-    QueryHandle qh = (QueryHandle) qHelper.executeQuery(QueryInventory.getSleepQuery("3"), null, session).getData();
-    sHelper.closeSession(session);
-    sHelper.closeSession(session);
-  }
-
-  //LENS-1199
-  @Test(enabled = true)
-  public void multipleCloseSession() throws Exception {
-    String session = sHelper.openSession("diff", "diff", lens.getCurrentDB());
-    sHelper.closeSession(session);
-    MapBuilder query = new MapBuilder("sessionid", session);
-    Response response = lens.exec("delete", SessionURL.SESSION_BASE_URL, servLens, null, query);
-    AssertUtil.assertGone(response);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/config/ITServerConfigTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/config/ITServerConfigTests.java b/lens-regression/src/test/java/org/apache/lens/regression/config/ITServerConfigTests.java
index 3503310..0f05198 100644
--- a/lens-regression/src/test/java/org/apache/lens/regression/config/ITServerConfigTests.java
+++ b/lens-regression/src/test/java/org/apache/lens/regression/config/ITServerConfigTests.java
@@ -80,18 +80,13 @@ public class ITServerConfigTests extends BaseTestClass {
   public void setUp(Method method) throws Exception {
     logger.info("Test Name: " + method.getName());
     Util.runRemoteCommand("cp " + confFilePath + " " + backupConfFilePath);
-    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
   }
 
-
   @AfterMethod(alwaysRun = true)
   public void restoreConfig() throws JSchException, IOException, LensException, InterruptedException {
     logger.info("Executing after method\n");
     Util.runRemoteCommand("cp " + backupConfFilePath + " " + confFilePath);
     lens.restart();
-    if (sessionHandleString != null){
-      sHelper.closeSession();
-    }
   }
 
 
@@ -131,7 +126,9 @@ public class ITServerConfigTests extends BaseTestClass {
 
   @DataProvider(name = "boolean_values")
   public Object[][] data() {
+//    String[][] testData = {{"true"}};
     String[][] testData = {{"true"}, {"false"}};
+
     return testData;
   }
 
@@ -289,47 +286,51 @@ public class ITServerConfigTests extends BaseTestClass {
   *  Test for Property lens.server.mode=METASTORE_READONLY,METASTORE_NODROP,OPEN
   */
 
-
   @Test(enabled = true)
   public void testServerMode() throws Exception {
 
-    String newDb = "TestMetastoreService_testDb1";
+    try{
+      sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+      String newDb = "TestMetastoreService_testDb1";
 
-    HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.SERVER_MODE, "METASTORE_READONLY");
-    Util.changeConfig(map, confFilePath);
-    lens.restart();
+      HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.SERVER_MODE, "METASTORE_READONLY");
+      Util.changeConfig(map, confFilePath);
+      lens.restart();
 
-    MapBuilder query = new MapBuilder("sessionid", sessionHandleString);
-    Response response = mHelper.exec("post", MetastoreURL.METASTORE_DATABASES_URL, servLens,
-        null, query, MediaType.APPLICATION_XML_TYPE, null, newDb);
-    Assert.assertEquals(response.getStatus(), Response.Status.METHOD_NOT_ALLOWED.getStatusCode());
+      MapBuilder query = new MapBuilder("sessionid", sessionHandleString);
+      Response response = mHelper.exec("post", MetastoreURL.METASTORE_DATABASES_URL, servLens,
+          null, query, MediaType.APPLICATION_XML_TYPE, null, newDb);
+      Assert.assertEquals(response.getStatus(), Response.Status.METHOD_NOT_ALLOWED.getStatusCode());
 
-    map.put(LensConfConstants.SERVER_MODE, "METASTORE_NODROP");
-    Util.changeConfig(map, confFilePath);
-    lens.restart();
+      map.put(LensConfConstants.SERVER_MODE, "METASTORE_NODROP");
+      Util.changeConfig(map, confFilePath);
+      lens.restart();
 
-    response = mHelper.exec("post", MetastoreURL.METASTORE_DATABASES_URL, servLens,
-        null, query, MediaType.APPLICATION_XML_TYPE, null, newDb);
-    AssertUtil.assertSucceededResponse(response);
-    StringList allDb = mHelper.listDatabases();
-    Assert.assertTrue(allDb.getElements().contains(newDb.toLowerCase()), "Unable to Create DB");
+      response = mHelper.exec("post", MetastoreURL.METASTORE_DATABASES_URL, servLens,
+          null, query, MediaType.APPLICATION_XML_TYPE, null, newDb);
+      AssertUtil.assertSucceededResponse(response);
+      StringList allDb = mHelper.listDatabases();
+      Assert.assertTrue(allDb.getElements().contains(newDb.toLowerCase()), "Unable to Create DB");
 
-    query.put("cascade", "true");
-    response = mHelper.exec("delete", MetastoreURL.METASTORE_DATABASES_URL + "/" + newDb, servLens,
-        null, query, MediaType.APPLICATION_XML_TYPE, null);
-    Assert.assertEquals(response.getStatus(), Response.Status.METHOD_NOT_ALLOWED.getStatusCode());
+      query.put("cascade", "true");
+      response = mHelper.exec("delete", MetastoreURL.METASTORE_DATABASES_URL + "/" + newDb, servLens,
+          null, query, MediaType.APPLICATION_XML_TYPE, null);
+      Assert.assertEquals(response.getStatus(), Response.Status.METHOD_NOT_ALLOWED.getStatusCode());
 
-    map.put(LensConfConstants.SERVER_MODE, "OPEN");
-    Util.changeConfig(map, confFilePath);
-    lens.restart();
+      map.put(LensConfConstants.SERVER_MODE, "OPEN");
+      Util.changeConfig(map, confFilePath);
+      lens.restart();
 
-    //TODO : Enable this when delete db issue is fixed
-/*    response = mHelper.exec("delete", MetastoreURL.METASTORE_DATABASES_URL + "/" + newDb, servLens,
-        null, query, MediaType.APPLICATION_XML_TYPE, null);
-    AssertUtil.assertSucceededResponse(response);
-    allDb = mHelper.listDatabases();
-    Assert.assertFalse(allDb.getElements().contains(newDb.toLowerCase()), "Unable to Create DB");*/
+      //TODO : Enable this when delete db issue is fixed
+      response = mHelper.exec("delete", MetastoreURL.METASTORE_DATABASES_URL + "/" + newDb, servLens,
+          null, query, MediaType.APPLICATION_XML_TYPE, null);
+      AssertUtil.assertSucceededResponse(response);
+      allDb = mHelper.listDatabases();
+      Assert.assertFalse(allDb.getElements().contains(newDb.toLowerCase()), "Unable to Create DB");
 
+    }finally {
+      sHelper.closeSession();
+    }
   }
 
 
@@ -390,36 +391,44 @@ public class ITServerConfigTests extends BaseTestClass {
   * Test for Property lens.server.max.finished.queries for persistent result set
   */
 
-  //This is failing
   @Test(enabled = true)
   public void testQueryResultRetention() throws Exception {
 
-    sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_SET, "true");
-    sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_INDRIVER, "false");
+    try{
+      sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+      sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_SET, "true");
+      sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_INDRIVER, "false");
 
-    HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.RESULTSET_PURGE_ENABLED, "true",
-        LensConfConstants.RESULTSET_PURGE_INTERVAL_IN_SECONDS, "10",
-        LensConfConstants.QUERY_RESULTSET_RETENTION, "20 sec",
-        LensConfConstants.HDFS_OUTPUT_RETENTION, "60 min");
+      HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.RESULTSET_PURGE_ENABLED, "true",
+          LensConfConstants.RESULTSET_PURGE_INTERVAL_IN_SECONDS, "10",
+          LensConfConstants.QUERY_RESULTSET_RETENTION, "20 sec",
+          LensConfConstants.HDFS_OUTPUT_RETENTION, "60 min");
 
-    Util.changeConfig(map, confFilePath);
-    lens.restart();
+      Util.changeConfig(map, confFilePath);
+      lens.restart();
 
-    MapBuilder query = new MapBuilder("sessionid", sessionHandleString);
+      MapBuilder query = new MapBuilder("sessionid", sessionHandleString);
+
+      QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(QueryInventory.JDBC_CUBE_QUERY).getData();
+      LensQuery lensQuery = qHelper.waitForCompletion(queryHandle);
+      Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
 
-    QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(QueryInventory.JDBC_CUBE_QUERY).getData();
-    LensQuery lensQuery = qHelper.waitForCompletion(queryHandle);
-    Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+      Response response = qHelper.exec("get", QueryURL.QUERY_URL + "/" + queryHandle.toString() + "/resultset",
+          servLens, null, query);
+      AssertUtil.assertSucceededResponse(response);
 
-    Response response = qHelper.exec("get", QueryURL.QUERY_URL + "/" + queryHandle.toString() + "/resultset",
-        servLens, null, query);
-    AssertUtil.assertSucceededResponse(response);
+      Thread.sleep(40000);
 
-    Thread.sleep(40000);
+      response = qHelper.exec("get", QueryURL.QUERY_URL + "/" + queryHandle.toString() + "/resultset",
+          servLens, null, query);
+      //TODO : This is failing , enable it when its fixed
+//    Assert.assertEquals(response.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
 
-    response = qHelper.exec("get", QueryURL.QUERY_URL + "/" + queryHandle.toString() + "/resultset",
-        servLens, null, query);
-    Assert.assertEquals(response.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
+    } finally {
+      if (sessionHandleString != null){
+        sHelper.closeSession();
+      }
+    }
   }
 
 
@@ -430,52 +439,67 @@ public class ITServerConfigTests extends BaseTestClass {
   @Test(enabled = true, dataProvider = "query_provider")
   public void testInMemoryPurger(String query) throws Exception {
 
-    sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_SET, "false");
-    sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_INDRIVER, "false");
-    sHelper.setAndValidateParam(LensConfConstants.QUERY_MAIL_NOTIFY, "false");
+    try{
+      sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+      sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_SET, "false");
+      sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_INDRIVER, "false");
+      sHelper.setAndValidateParam(LensConfConstants.QUERY_MAIL_NOTIFY, "false");
 
-    HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.INMEMORY_RESULT_SET_TTL_SECS, "20",
-        LensConfConstants.PURGE_INTERVAL, "10000"); //in millis
-    Util.changeConfig(map, confFilePath);
-    lens.restart();
-    QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(query).getData();
-    LensQuery lensQuery = qHelper.waitForCompletion(queryHandle);
-    Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+      HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.INMEMORY_RESULT_SET_TTL_SECS, "20",
+          LensConfConstants.PURGE_INTERVAL, "10000"); //in millis
+      Util.changeConfig(map, confFilePath);
+      lens.restart();
+      QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(query).getData();
+      LensQuery lensQuery = qHelper.waitForCompletion(queryHandle);
+      Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
 
-    Response response = qHelper.getResultSetResponse(queryHandle, "0", "100", sessionHandleString);
-    Assert.assertEquals(response.getStatus(), Response.Status.OK.getStatusCode());
+      Response response = qHelper.getResultSetResponse(queryHandle, "0", "100", sessionHandleString);
+      Assert.assertEquals(response.getStatus(), Response.Status.OK.getStatusCode());
 
-    Thread.sleep(30000); //waiting till query gets purged ( ttl + purge interval time)
+      Thread.sleep(30000); //waiting till query gets purged ( ttl + purge interval time)
 
-    response = qHelper.getResultSetResponse(queryHandle, "0", "100", sessionHandleString);
-    Assert.assertEquals(response.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
+      response = qHelper.getResultSetResponse(queryHandle, "0", "100", sessionHandleString);
+      Assert.assertEquals(response.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
+    } finally {
+      if (sessionHandleString != null){
+        sHelper.closeSession();
+      }
+    }
   }
 
   // Behaviour is not the same for hive query before result is purged
   @Test(enabled = true)
   public void readInmemoryTwiceBeforePurgerTime() throws Exception {
 
-    sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_SET, "false");
-    sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_INDRIVER, "false");
-    sHelper.setAndValidateParam(LensConfConstants.QUERY_MAIL_NOTIFY, "false");
+    try{
+      sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+      sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_SET, "false");
+      sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_INDRIVER, "false");
+      sHelper.setAndValidateParam(LensConfConstants.QUERY_MAIL_NOTIFY, "false");
 
-    HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.INMEMORY_RESULT_SET_TTL_SECS, "500",
-        LensConfConstants.PURGE_INTERVAL, "10000");
-    Util.changeConfig(map, confFilePath);
-    lens.restart();
+      HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.INMEMORY_RESULT_SET_TTL_SECS, "500",
+          LensConfConstants.PURGE_INTERVAL, "10000");
+      Util.changeConfig(map, confFilePath);
+      lens.restart();
 
-    QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(QueryInventory.JDBC_CUBE_QUERY).getData();
-    LensQuery lensQuery = qHelper.waitForCompletion(queryHandle);
-    Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL, "Query did not succeed");
+      QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(QueryInventory.JDBC_CUBE_QUERY).getData();
+      LensQuery lensQuery = qHelper.waitForCompletion(queryHandle);
+      Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL, "Query did not succeed");
 
-    Response response = qHelper.getResultSetResponse(queryHandle, "0", "100", sessionHandleString);
-    Assert.assertEquals(response.getStatus(), Response.Status.OK.getStatusCode());
+      Response response = qHelper.getResultSetResponse(queryHandle, "0", "100", sessionHandleString);
+      Assert.assertEquals(response.getStatus(), Response.Status.OK.getStatusCode());
 
-    response = qHelper.getResultSetResponse(queryHandle, "0", "100", sessionHandleString);
+      response = qHelper.getResultSetResponse(queryHandle, "0", "100", sessionHandleString);
 
-    //TODO : enable this when LENS-823 is fixed
-    //Currently its throwing 500 which needs to be fixed.
+      //TODO : enable this when LENS-823 is fixed
+      //Currently its throwing 500 which needs to be fixed.
 //    Assert.assertEquals(response.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
+
+    } finally {
+      if (sessionHandleString != null){
+        sHelper.closeSession();
+      }
+    }
   }
 
 
@@ -553,8 +577,5 @@ public class ITServerConfigTests extends BaseTestClass {
       }
     }
   }
-
-
-
 }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/config/ITSessionConfigTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/config/ITSessionConfigTests.java b/lens-regression/src/test/java/org/apache/lens/regression/config/ITSessionConfigTests.java
index 2deb043..fcced5e 100644
--- a/lens-regression/src/test/java/org/apache/lens/regression/config/ITSessionConfigTests.java
+++ b/lens-regression/src/test/java/org/apache/lens/regression/config/ITSessionConfigTests.java
@@ -28,7 +28,7 @@ import javax.xml.bind.JAXBException;
 
 import org.apache.lens.api.query.*;
 import org.apache.lens.cube.parse.CubeQueryConfUtil;
-import org.apache.lens.regression.client.SessionResourceTests;
+import org.apache.lens.regression.client.ITSessionResourceTests;
 import org.apache.lens.regression.core.constants.QueryInventory;
 import org.apache.lens.regression.core.helpers.ServiceManagerHelper;
 import org.apache.lens.regression.core.testHelper.BaseTestClass;
@@ -52,7 +52,7 @@ public class ITSessionConfigTests extends BaseTestClass{
   private static String queryResultParentDirPath = "/tmp/lensreports";
   private String lensConfFilePath = lens.getServerDir() + "/conf/lens-site.xml";
 
-  private static Logger logger = Logger.getLogger(SessionResourceTests.class);
+  private static Logger logger = Logger.getLogger(ITSessionResourceTests.class);
 
   @BeforeClass(alwaysRun = true)
   public void initialize() throws IOException, JSchException, JAXBException, LensException {

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/scheduler/ITMaxScheduledQueryTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/scheduler/ITMaxScheduledQueryTests.java b/lens-regression/src/test/java/org/apache/lens/regression/scheduler/ITMaxScheduledQueryTests.java
new file mode 100644
index 0000000..5c4870a
--- /dev/null
+++ b/lens-regression/src/test/java/org/apache/lens/regression/scheduler/ITMaxScheduledQueryTests.java
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.lens.regression.scheduler;
+
+import java.lang.reflect.Method;
+import java.util.Calendar;
+import java.util.HashMap;
+
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.Response;
+
+import org.apache.lens.api.scheduler.*;
+import org.apache.lens.regression.core.constants.QueryInventory;
+import org.apache.lens.regression.core.helpers.ServiceManagerHelper;
+import org.apache.lens.regression.core.testHelper.BaseTestClass;
+import org.apache.lens.regression.util.AssertUtil;
+import org.apache.lens.regression.util.Util;
+import org.apache.lens.server.api.LensConfConstants;
+import org.apache.lens.server.api.util.LensUtil;
+
+import org.apache.log4j.Logger;
+
+import org.testng.Assert;
+import org.testng.annotations.*;
+
+
+public class ITMaxScheduledQueryTests extends BaseTestClass {
+
+  WebTarget servLens;
+  private String sessionHandleString;
+
+  private static Logger logger = Logger.getLogger(ITMaxScheduledQueryTests.class);
+  private static String format = "yyyy-MM-dd HH:mm:ss";
+  private static String currentDate = Util.getCurrentDate(format);
+  String lensSiteConf = lens.getServerDir() + "/conf/lens-site.xml";
+
+  @BeforeClass(alwaysRun = true)
+  public void initialize() throws Exception {
+    servLens = ServiceManagerHelper.init();
+    HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.MAX_SCHEDULED_JOB_PER_USER, "2");
+    Util.changeConfig(map, lensSiteConf);
+    lens.restart();
+  }
+
+  @BeforeMethod(alwaysRun = true)
+  public void setUp(Method method) throws Exception {
+    logger.info("Test Name: " + method.getName());
+    logger.info("Creating a new Session");
+    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+  }
+
+  @AfterMethod(alwaysRun = true)
+  public void closeSession() throws Exception {
+    logger.info("Closing Session");
+    if (sessionHandleString != null){
+      sHelper.closeSession();
+    }
+  }
+
+  @AfterClass(alwaysRun = true)
+  public void afterClass() throws Exception {
+    Util.changeConfig(lensSiteConf);
+    lens.restart();
+  }
+
+  //LENS-1320
+
+  @Test(groups = "max_scheduled_job_per_user")
+  public void testJobsInNewState() throws Exception {
+
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 4);
+    String session = sHelper.openSession("max1", "pwd", lens.getCurrentDB());
+    XJob xJob = scheduleHelper.getXJob("job-submit", QueryInventory.QUERY, null, currentDate, endDate,
+        XFrequencyEnum.DAILY);
+
+    String j1 = scheduleHelper.submitJob(xJob, session);
+    String j2 = scheduleHelper.submitJob(xJob, session);
+    Response response = scheduleHelper.submitJobReturnResponse("submit", xJob, session);
+    AssertUtil.assertBadRequest(response);
+  }
+
+  @Test(groups = "max_scheduled_job_per_user")
+  public void testJobsInScheduledState() throws Exception {
+
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 4);
+    String session = sHelper.openSession("max2", "pwd", lens.getCurrentDB());
+    XJob xJob = scheduleHelper.getXJob("job-submit", QueryInventory.QUERY, null, currentDate, endDate,
+        XFrequencyEnum.DAILY);
+
+    String j1 = scheduleHelper.submitJob(xJob, session);
+    String j2 = scheduleHelper.submitNScheduleJob(xJob, session);
+    Response response = scheduleHelper.submitJobReturnResponse("submit", xJob, session);
+    AssertUtil.assertBadRequest(response);
+  }
+
+  @Test(groups = "max_scheduled_job_per_user")
+  public void testJobsInSuspendedState() throws Exception {
+
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 4);
+    String session = sHelper.openSession("max3", "pwd", lens.getCurrentDB());
+    XJob xJob = scheduleHelper.getXJob("job-submit", QueryInventory.QUERY, null, currentDate, endDate,
+        XFrequencyEnum.DAILY);
+
+    String j1 = scheduleHelper.submitJob(xJob, session);
+    String j2 = scheduleHelper.submitNScheduleJob(xJob, session);
+    scheduleHelper.updateJob(j2, "SUSPEND", session);
+    Assert.assertEquals(scheduleHelper.getJobStatus(j2), SchedulerJobState.SUSPENDED);
+    Response response = scheduleHelper.submitJobReturnResponse("submit", xJob, session);
+    AssertUtil.assertBadRequest(response);
+  }
+
+  @Test(groups = "max_scheduled_job_per_user")
+  public void testJobsInDeletedState() throws Exception {
+
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 4);
+    String session = sHelper.openSession("max4", "pwd", lens.getCurrentDB());
+    XJob xJob = scheduleHelper.getXJob("job-submit", QueryInventory.QUERY, null, currentDate, endDate,
+        XFrequencyEnum.DAILY);
+
+    String j1 = scheduleHelper.submitJob(xJob, session);
+    scheduleHelper.deleteJob(j1, session);
+    Assert.assertEquals(scheduleHelper.getJobStatus(j1), SchedulerJobState.DELETED);
+
+    String j2 = scheduleHelper.submitJob(xJob, session);
+    String j3 = scheduleHelper.submitJob(xJob, session);
+  }
+
+  @Test(groups = "max_scheduled_job_per_user")
+  public void testJobsInExpiredState() throws Exception {
+
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 4);
+    String session = sHelper.openSession("max5", "pwd", lens.getCurrentDB());
+    XJob xJob = scheduleHelper.getXJob("job-submit", QueryInventory.QUERY, null, currentDate, endDate,
+        XFrequencyEnum.DAILY);
+
+    String j1 = scheduleHelper.submitJob(xJob, session);
+    scheduleHelper.updateJob(j1, "EXPIRE", session);
+    Assert.assertEquals(scheduleHelper.getJobStatus(j1), SchedulerJobState.EXPIRED);
+    String j2 = scheduleHelper.submitJob(xJob, session);
+    String j3 = scheduleHelper.submitJob(xJob, session);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/scheduler/ITScheduleQueryTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/scheduler/ITScheduleQueryTests.java b/lens-regression/src/test/java/org/apache/lens/regression/scheduler/ITScheduleQueryTests.java
new file mode 100644
index 0000000..f8fa33f
--- /dev/null
+++ b/lens-regression/src/test/java/org/apache/lens/regression/scheduler/ITScheduleQueryTests.java
@@ -0,0 +1,337 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.lens.regression.scheduler;
+
+import java.lang.reflect.Method;
+import java.util.Calendar;
+import java.util.List;
+
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.lens.api.result.LensAPIResult;
+import org.apache.lens.api.scheduler.*;
+import org.apache.lens.regression.core.constants.QueryInventory;
+import org.apache.lens.regression.core.helpers.ServiceManagerHelper;
+import org.apache.lens.regression.core.testHelper.BaseTestClass;
+import org.apache.lens.regression.util.Util;
+
+import org.apache.log4j.Logger;
+
+import org.testng.Assert;
+import org.testng.annotations.*;
+
+
+public class ITScheduleQueryTests extends BaseTestClass {
+
+  WebTarget servLens;
+  private String sessionHandleString;
+
+  private static Logger logger = Logger.getLogger(ITScheduleQueryTests.class);
+  private static String format = "yyyy-MM-dd HH:mm:ss";
+  private static String currentDate = Util.getCurrentDate(format);
+  String lensSiteConf = lens.getServerDir() + "/conf/lens-site.xml";
+
+  @BeforeClass(alwaysRun = true)
+  public void initialize() throws Exception {
+    servLens = ServiceManagerHelper.init();
+  }
+
+  @BeforeMethod(alwaysRun = true)
+  public void setUp(Method method) throws Exception {
+    logger.info("Test Name: " + method.getName());
+    logger.info("Creating a new Session");
+    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+  }
+
+  @AfterMethod(alwaysRun = true)
+  public void closeSession() throws Exception {
+    logger.info("Closing Session");
+    if (sessionHandleString != null){
+      sHelper.closeSession();
+    }
+  }
+
+
+  @Test
+  public void submitJob() throws Exception {
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 4);
+    XJob xJob = scheduleHelper.getXJob("job-submit", QueryInventory.QUERY, null, currentDate, endDate,
+        XFrequencyEnum.DAILY);
+    String jobHandle = scheduleHelper.submitJob(xJob, sessionHandleString);
+    Assert.assertNotNull(jobHandle);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.NEW);
+  }
+
+  @Test
+  public void submitNScheduleQuery() throws Exception {
+
+    String startDate = Util.modifyDate(currentDate, format, Calendar.DATE, -1);
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 3);
+    XJob xJob = scheduleHelper.getXJob("job-submit-schedule", QueryInventory.JDBC_CUBE_QUERY, null, startDate,
+        endDate, XFrequencyEnum.DAILY);
+    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
+    Assert.assertNotNull(jobHandle);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SCHEDULED);
+
+    SchedulerJobInfo jobInfo = scheduleHelper.getJobDetails(jobHandle, sessionHandleString).getData();
+    Assert.assertNotNull(jobInfo);
+    Assert.assertEquals(jobInfo.getJob().getName(), "job-submit-schedule");
+  }
+
+  //submit and schedule and also get job definition
+  @Test
+  public void submitNScheduleQueryCronExp() throws Exception {
+
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 1);
+    XJob xJob = scheduleHelper.getXJob("job-submit-schedule-cronExp", QueryInventory.QUERY, null, currentDate,
+        endDate, "0/30 * * * * ?");
+    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
+    Assert.assertNotNull(jobHandle);
+
+    XJob job = scheduleHelper.getJobDefinition(jobHandle, sessionHandleString, MediaType.APPLICATION_XML_TYPE,
+        MediaType.APPLICATION_XML).getData();
+    Assert.assertNotNull(job);
+    Assert.assertEquals(job.getName(), "job-submit-schedule-cronExp");
+  }
+
+
+  @Test
+  public void testDeleteJob() throws Exception {
+
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 1);
+    XJob xJob = scheduleHelper.getXJob("job-delete", QueryInventory.QUERY, null, currentDate, endDate,
+         "0/30 * * * * ?");
+
+    //delete in submit state
+    String jobHandle = scheduleHelper.submitJob(xJob, sessionHandleString);
+    scheduleHelper.deleteJob(jobHandle, sessionHandleString);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.DELETED);
+
+    //delete in scheduled state
+    jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
+    scheduleHelper.deleteJob(jobHandle, sessionHandleString);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.DELETED);
+
+    //delete in suspended state
+    jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
+    scheduleHelper.updateJob(jobHandle, "SUSPEND", sessionHandleString);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SUSPENDED);
+    scheduleHelper.deleteJob(jobHandle, sessionHandleString);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.DELETED);
+
+    //delete in expired state
+    jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
+    scheduleHelper.updateJob(jobHandle, "EXPIRE", sessionHandleString);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.EXPIRED);
+    scheduleHelper.deleteJob(jobHandle, sessionHandleString);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.DELETED);
+  }
+
+  @Test
+  public void testUpdateJob() throws Exception {
+
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 4);
+    XJob job = scheduleHelper.getXJob("job-update", QueryInventory.QUERY, null, currentDate, endDate,
+        XFrequencyEnum.WEEKLY);
+    String jobHandle = scheduleHelper.submitJob(job, sessionHandleString);
+
+    XJob tmp = scheduleHelper.getJobDefinition(jobHandle, sessionHandleString).getData();
+    tmp.setName("modified-name");
+    endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 6);
+    tmp.setEndTime(Util.getGregorianCalendar(endDate));
+    scheduleHelper.updateJob(tmp, jobHandle, sessionHandleString);
+
+    XJob modifiedJob = scheduleHelper.getJobDefinition(jobHandle, sessionHandleString).getData();
+    Assert.assertEquals(modifiedJob.getName(), "modified-name");
+    String modifiedEndTime = Util.getDateStringFromGregorainCalender(modifiedJob.getEndTime(), format);
+    Assert.assertEquals(modifiedEndTime, endDate);
+  }
+
+  @Test
+  public void testUpdateJobAction() throws Exception {
+
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 1);
+    XJob job = scheduleHelper.getXJob("job-update-action", QueryInventory.QUERY, null, currentDate, endDate,
+        "0/20 * * * * ?");
+    String jobHandle = scheduleHelper.submitJob(job, sessionHandleString);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.NEW);
+
+    scheduleHelper.updateJob(jobHandle, "SCHEDULE", sessionHandleString);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SCHEDULED);
+
+    scheduleHelper.updateJob(jobHandle, "SUSPEND", sessionHandleString);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SUSPENDED);
+
+    scheduleHelper.updateJob(jobHandle, "RESUME", sessionHandleString);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SCHEDULED);
+
+    scheduleHelper.updateJob(jobHandle, "EXPIRE", sessionHandleString);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.EXPIRED);
+  }
+
+  @Test
+  public void testGetAllInstancesOfAJob() throws Exception {
+
+    String startDate = Util.modifyDate(Util.getCurrentDate(format), format, Calendar.SECOND, 5);
+    String endDate = Util.modifyDate(startDate, format, Calendar.MINUTE, 4);
+    XJob xJob = scheduleHelper.getXJob("job-update-action", QueryInventory.QUERY, null, startDate, endDate,
+        "0/20 * * * * ?");
+    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
+
+    Thread.sleep(60000);
+
+    List<SchedulerJobInstanceInfo> instanceList = scheduleHelper.getAllInstancesOfJob(jobHandle, "10",
+        sessionHandleString);
+    Assert.assertEquals(instanceList.size(), 3);
+  }
+
+
+  @Test
+  public void updateInstance() throws Exception {
+
+    String startDate = Util.modifyDate(Util.getCurrentDate(format), format, Calendar.SECOND, 10);
+    String endDate = Util.modifyDate(startDate, format, Calendar.MINUTE, 3);
+    XJob xJob = scheduleHelper.getXJob("job-update-action", QueryInventory.JDBC_DIM_QUERY, null, startDate, endDate,
+        "0/20 * * * * ?");
+    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
+
+    Thread.sleep(20000);
+
+    List<SchedulerJobInstanceInfo> instanceList = scheduleHelper.getAllInstancesOfJob(jobHandle, "10",
+        sessionHandleString);
+
+    Thread.sleep(10000);
+
+    LensAPIResult res = scheduleHelper.updateInstance(instanceList.get(0).getId().getHandleIdString(),
+        "RERUN", sessionHandleString);
+    Assert.assertTrue(res.getData().equals(true));
+
+    SchedulerJobInstanceInfo instanceInfo = scheduleHelper.getInstanceDetails(instanceList.get(0).getId()
+        .getHandleIdString(), sessionHandleString).getData();
+    List<SchedulerJobInstanceRun> runList = instanceInfo.getInstanceRunList();
+    Assert.assertEquals(runList.size(), 2);
+    Assert.assertEquals(runList.get(1).getRunId(), 2);
+  }
+
+
+  @Test(enabled = true)
+  public void restart() throws Exception {
+
+    String startDate = Util.modifyDate(Util.getCurrentDate(format), format, Calendar.SECOND, 5);
+    String endDate = Util.modifyDate(startDate, format, Calendar.MINUTE, 2);
+    XJob xJob = scheduleHelper.getXJob("job-restart", QueryInventory.QUERY, null, startDate, endDate, "0/20 * * * * ?");
+
+    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
+    Assert.assertNotNull(jobHandle);
+
+    Thread.sleep(20000);
+    lens.stop();
+    Thread.sleep(20000);
+    lens.start();
+    Thread.sleep(60000);
+
+    List<SchedulerJobInstanceInfo> instanceList = scheduleHelper.getAllInstancesOfJob(jobHandle, "50",
+        sessionHandleString);
+    Assert.assertEquals(instanceList.size(), 6);
+  }
+
+  //LENS
+  @Test
+  public void testMisfiredEvents() throws Exception {
+
+    String startDate = Util.modifyDate(currentDate, format, Calendar.DATE, -2);
+    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 3);
+    XJob xJob = scheduleHelper.getXJob("job-misfire", QueryInventory.JDBC_CUBE_QUERY, null, startDate,
+        endDate, XFrequencyEnum.DAILY);
+    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
+    Assert.assertNotNull(jobHandle);
+    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SCHEDULED);
+
+    List<SchedulerJobInstanceInfo> instanceList = scheduleHelper.getAllInstancesOfJob(jobHandle, "10",
+        sessionHandleString);
+    Assert.assertEquals(instanceList.size(), 3);
+
+    if (jobHandle!=null){
+      scheduleHelper.updateJob(jobHandle, "EXPIRE", sessionHandleString);
+    }
+  }
+
+
+  //LENS-1286
+  @Test
+  public void testRunningInstanceOnRestart() throws Exception {
+
+    String startDate = Util.modifyDate(Util.getCurrentDate(format), format, Calendar.SECOND, 2);
+    String endDate = Util.modifyDate(startDate, format, Calendar.SECOND, 15);
+    XJob xJob = scheduleHelper.getXJob("job-restart", QueryInventory.getSleepQuery("5"), null, startDate, endDate,
+        "0/10 * * * * ?");
+    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
+
+    Thread.sleep(10000);
+
+    List<SchedulerJobInstanceInfo> instanceList = scheduleHelper.getAllInstancesOfJob(jobHandle, "10",
+        sessionHandleString);
+
+    lens.restart();
+
+    SchedulerJobInstanceInfo instanceInfo = scheduleHelper.getInstanceDetails(instanceList.get(0).getId()
+        .getHandleIdString(), sessionHandleString).getData();
+
+    SchedulerJobInstanceRun instanceRun = instanceInfo.getInstanceRunList().get(0);
+    qHelper.waitForCompletion(instanceRun.getQueryHandle());
+    Assert.assertEquals(instanceRun.getInstanceState(), SchedulerJobInstanceState.SUCCEEDED);
+
+  }
+
+  @Test(enabled = false)
+  public void testQueryNotFoundCaseOnRestart() throws Exception {
+
+    String startDate = Util.modifyDate(Util.getCurrentDate(format), format, Calendar.SECOND, 2);
+    String endDate = Util.modifyDate(startDate, format, Calendar.SECOND, 15);
+    XJob xJob = scheduleHelper.getXJob("job-restart", QueryInventory.getSleepQuery("5"), null, startDate, endDate,
+        "0/10 * * * * ?");
+    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
+
+    Thread.sleep(10000);
+
+    List<SchedulerJobInstanceInfo> instanceList = scheduleHelper.getAllInstancesOfJob(jobHandle, "10",
+        sessionHandleString);
+
+    lens.stop();
+    Util.runRemoteCommand("hadoop dfs -rmr /tmp/lensserver/query.*");
+    lens.start();
+
+    Thread.sleep(15000); // wait till instance gets killed and new instance is spawned
+
+    String firstInstanceHandle = instanceList.get(0).getId().getHandleIdString();
+    SchedulerJobInstanceInfo instanceInfo = scheduleHelper.getInstanceDetails(firstInstanceHandle, sessionHandleString)
+        .getData();
+
+    Assert.assertEquals(instanceInfo.getInstanceRunList().get(0).getInstanceState(), SchedulerJobInstanceState.KILLED);
+    qHelper.waitForCompletion(instanceInfo.getInstanceRunList().get(1).getQueryHandle());
+    Thread.sleep(3000);
+
+    instanceInfo = scheduleHelper.getInstanceDetails(firstInstanceHandle, sessionHandleString).getData();
+    Assert.assertEquals(instanceInfo.getInstanceRunList().get(1).getInstanceState(),
+        SchedulerJobInstanceState.SUCCEEDED);
+  }
+
+}


[09/16] lens git commit: LENS-1380 : Revamp testcase division

Posted by pr...@apache.org.
LENS-1380 : Revamp testcase division


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

Branch: refs/heads/lens-1381
Commit: 97fe26d3ca0db9b383c8d86feb6bb7b00cacdc7e
Parents: 6af57fb
Author: Archana H <ar...@apache.org>
Authored: Mon Feb 6 13:16:19 2017 +0530
Committer: Archana H <ar...@apache.org>
Committed: Mon Feb 6 13:16:19 2017 +0530

----------------------------------------------------------------------
 .../regression/core/constants/DriverConfig.java |   2 -
 .../core/helpers/ScheduleResourceHelper.java    |  62 +-
 .../apache/lens/regression/util/AssertUtil.java |   1 -
 .../src/main/resources/template.lens.properties |   9 +-
 .../apache/lens/regression/ITSessionTests.java  | 163 +++++
 .../apache/lens/regression/SessionTests.java    | 163 -----
 .../client/ITDuplicateQueryTests.java           | 188 ++++++
 .../regression/client/ITKillQueryTests.java     | 361 +++++++++++
 .../lens/regression/client/ITListQueryTest.java |   7 +-
 .../regression/client/ITPreparedQueryTests.java |  13 +-
 .../lens/regression/client/ITQueryApiTests.java | 182 ++----
 .../regression/client/ITScheduleQueryTests.java | 284 ---------
 .../client/ITSessionResourceTests.java          | 401 ++++++++++++
 .../lens/regression/client/KillQueryTests.java  | 362 -----------
 .../regression/client/SessionResourceTests.java | 403 ------------
 .../regression/config/ITServerConfigTests.java  | 197 +++---
 .../regression/config/ITSessionConfigTests.java |   4 +-
 .../scheduler/ITMaxScheduledQueryTests.java     | 160 +++++
 .../scheduler/ITScheduleQueryTests.java         | 337 +++++++++++
 .../lens/regression/throttling/ITCostTests.java | 176 +-----
 .../throttling/ITQueueNumberTests.java          | 232 +++++++
 .../throttling/ITThrottlingTests.java           | 605 +++++++++++++++++++
 .../lens/regression/throttling/Throttling.java  | 604 ------------------
 23 files changed, 2680 insertions(+), 2236 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/main/java/org/apache/lens/regression/core/constants/DriverConfig.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/main/java/org/apache/lens/regression/core/constants/DriverConfig.java b/lens-regression/src/main/java/org/apache/lens/regression/core/constants/DriverConfig.java
index cff8e91..5201272 100644
--- a/lens-regression/src/main/java/org/apache/lens/regression/core/constants/DriverConfig.java
+++ b/lens-regression/src/main/java/org/apache/lens/regression/core/constants/DriverConfig.java
@@ -20,7 +20,6 @@
 package org.apache.lens.regression.core.constants;
 
 import org.apache.lens.driver.jdbc.JDBCDriverConfConstants;
-import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.query.constraint.MaxConcurrentDriverQueriesConstraintFactory;
 import org.apache.lens.server.query.constraint.TotalQueryCostCeilingConstraintFactory;
 
@@ -38,7 +37,6 @@ public class DriverConfig {
       MAX_CONCURRENT_QUERIES_PER_QUEUE_KEY;
   public static final String JDBC_POOL_SIZE = JDBCDriverConfConstants.ConnectionPoolProperties.
       JDBC_POOL_MAX_SIZE.getConfigKey();
-  public static final String HIVE_CONSTRAINT_FACTORIES = LensConfConstants.QUERY_LAUNCHING_CONSTRAINT_FACTORIES_SFX;
 
 
   public static final String MAX_CONCURRENT_CONSTRAINT_FACTORY = MaxConcurrentDriverQueriesConstraintFactory

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/main/java/org/apache/lens/regression/core/helpers/ScheduleResourceHelper.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/main/java/org/apache/lens/regression/core/helpers/ScheduleResourceHelper.java b/lens-regression/src/main/java/org/apache/lens/regression/core/helpers/ScheduleResourceHelper.java
index f2883f0..97e1daf 100644
--- a/lens-regression/src/main/java/org/apache/lens/regression/core/helpers/ScheduleResourceHelper.java
+++ b/lens-regression/src/main/java/org/apache/lens/regression/core/helpers/ScheduleResourceHelper.java
@@ -28,8 +28,8 @@ import javax.xml.bind.JAXBElement;
 import javax.xml.bind.JAXBException;
 import javax.xml.datatype.DatatypeConfigurationException;
 
-import org.apache.lens.api.APIResult;
 import org.apache.lens.api.ToXMLString;
+import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.api.scheduler.*;
 import org.apache.lens.regression.core.type.MapBuilder;
 import org.apache.lens.regression.util.AssertUtil;
@@ -51,8 +51,7 @@ public class ScheduleResourceHelper extends ServiceManagerHelper {
     super(envFileName);
   }
 
-
-  public String submitJob(String action, XJob job, String sessionHandleString)
+  public Response submitJobReturnResponse(String action, XJob job, String sessionHandleString)
     throws JAXBException, IOException, ParseException, DatatypeConfigurationException {
 
     MapBuilder map = new MapBuilder("sessionid", sessionHandleString);
@@ -64,19 +63,27 @@ public class ScheduleResourceHelper extends ServiceManagerHelper {
     GenericEntity<JAXBElement<XJob>> entry = new GenericEntity<JAXBElement<XJob>>(xmlJob){};
     Response response = this.exec("post", SCHEDULER_JOBS_URL, servLens, null, map, MediaType.APPLICATION_XML_TYPE,
         MediaType.APPLICATION_XML, entry);
+    return  response;
+  }
+
+  public String submitJob(String action, XJob job, String session)
+    throws JAXBException, IOException, ParseException, DatatypeConfigurationException {
+
+    Response response = submitJobReturnResponse(action, job, session);
     AssertUtil.assertSucceededResponse(response);
-    SchedulerJobHandle handle =  response.readEntity(SchedulerJobHandle.class);
-    return handle.getHandleIdString();
+    LensAPIResult<SchedulerJobHandle> handle =  response.readEntity(
+        new GenericType<LensAPIResult<SchedulerJobHandle>>(){});
+    return handle.getData().getHandleIdString();
   }
 
-  public String submitJob(XJob job, String sessionHandleString)
+  public String submitJob(XJob job, String session)
     throws JAXBException, IOException, ParseException, DatatypeConfigurationException {
-    return submitJob("submit", job, sessionHandleString);
+    return submitJob("SUBMIT", job, session);
   }
 
-  public String submitNScheduleJob(XJob job, String sessionHandleString)
+  public String submitNScheduleJob(XJob job, String session)
     throws JAXBException, IOException, ParseException, DatatypeConfigurationException {
-    return submitJob("submit_and_schedule", job, sessionHandleString);
+    return submitJob("SUBMIT_AND_SCHEDULE", job, session);
   }
 
   public XJob getXJob(String name, String query, String db, String startTime, String endTime, XFrequencyEnum frequency)
@@ -148,27 +155,28 @@ public class ScheduleResourceHelper extends ServiceManagerHelper {
     return execution;
   }
 
-  public XJob getJobDefinition(String jobHandle, String sessionId, MediaType inputMedia, String outputMedia){
+  public LensAPIResult<XJob> getJobDefinition(String jobHandle, String sessionId, MediaType inputMedia,
+      String outputMedia){
     MapBuilder map = new MapBuilder("sessionid", sessionId);
     Response response = this.exec("get", SCHEDULER_JOBS_URL + "/" + jobHandle , servLens, null, map, inputMedia,
         outputMedia);
     AssertUtil.assertSucceededResponse(response);
-    return response.readEntity(XJob.class);
+    return response.readEntity(new GenericType<LensAPIResult<XJob>>(){});
   }
 
-  public XJob getJobDefinition(String jobHandle, String sessionId){
+  public LensAPIResult<XJob> getJobDefinition(String jobHandle, String sessionId){
     return getJobDefinition(jobHandle, sessionId, MediaType.APPLICATION_XML_TYPE, MediaType.APPLICATION_XML);
   }
 
-  public APIResult deleteJob(String jobHandle, String sessionId){
+  public LensAPIResult deleteJob(String jobHandle, String sessionId){
     MapBuilder map = new MapBuilder("sessionid", sessionId);
     Response response = this.exec("delete", SCHEDULER_JOBS_URL + "/" + jobHandle , servLens, null, map, null,
-        MediaType.APPLICATION_JSON);
+        MediaType.APPLICATION_XML);
     AssertUtil.assertSucceededResponse(response);
-    return response.readEntity(APIResult.class);
+    return response.readEntity(LensAPIResult.class);
   }
 
-  public APIResult updateJob(XJob job, String jobHandle, String sessionHandleString)
+  public LensAPIResult updateJob(XJob job, String jobHandle, String sessionHandleString)
     throws JAXBException, IOException, ParseException, DatatypeConfigurationException {
 
     MapBuilder map = new MapBuilder("sessionid", sessionHandleString);
@@ -177,35 +185,35 @@ public class ScheduleResourceHelper extends ServiceManagerHelper {
     Response response = this.exec("put", SCHEDULER_JOBS_URL + "/" + jobHandle, servLens, null, map,
         MediaType.APPLICATION_XML_TYPE, MediaType.APPLICATION_XML, ToXMLString.toString(xmlJob));
     AssertUtil.assertSucceededResponse(response);
-    return response.readEntity(APIResult.class);
+    return response.readEntity(LensAPIResult.class);
   }
 
-  public APIResult updateJob(String jobHandle, String action, String sessionHandleString)
+  public LensAPIResult updateJob(String jobHandle, String action, String sessionHandleString)
     throws JAXBException, IOException, ParseException, DatatypeConfigurationException {
 
     MapBuilder map = new MapBuilder("sessionid", sessionHandleString, "action", action);
     Response response = this.exec("post", SCHEDULER_JOBS_URL + "/" + jobHandle, servLens, null, map);
     AssertUtil.assertSucceededResponse(response);
-    return response.readEntity(APIResult.class);
+    return response.readEntity(LensAPIResult.class);
   }
 
-  public SchedulerJobInfo getJobDetails(String jobHandle, String sessionHandleString){
+  public LensAPIResult<SchedulerJobInfo> getJobDetails(String jobHandle, String sessionHandleString){
 
     MapBuilder map = new MapBuilder("sessionid", sessionHandleString);
     Response response = this.exec("get", SCHEDULER_JOBS_URL + "/" + jobHandle + "/info", servLens, null, map,
         MediaType.APPLICATION_XML_TYPE, MediaType.APPLICATION_XML);
     AssertUtil.assertSucceededResponse(response);
-    return response.readEntity(SchedulerJobInfo.class);
+    return response.readEntity(new GenericType<LensAPIResult<SchedulerJobInfo>>(){});
   }
 
 
   public SchedulerJobState getJobStatus(String jobHandle, String sessionHandleString){
-    SchedulerJobInfo jobInfo = getJobDetails(jobHandle, sessionHandleString);
+    SchedulerJobInfo jobInfo = getJobDetails(jobHandle, sessionHandleString).getData();
     return jobInfo.getJobState();
   }
 
   public SchedulerJobState getJobStatus(String jobHandle){
-    SchedulerJobInfo jobInfo = getJobDetails(jobHandle, sessionHandleString);
+    SchedulerJobInfo jobInfo = getJobDetails(jobHandle, sessionHandleString).getData();
     return jobInfo.getJobState();
   }
 
@@ -218,21 +226,21 @@ public class ScheduleResourceHelper extends ServiceManagerHelper {
     return response.readEntity(new GenericType<List<SchedulerJobInstanceInfo>>(){});
   }
 
-  public SchedulerJobInstanceInfo getInstanceDetails(String instanceHandle, String sessionId)
+  public LensAPIResult<SchedulerJobInstanceInfo> getInstanceDetails(String instanceHandle, String sessionId)
     throws JAXBException, IOException, ParseException, DatatypeConfigurationException {
 
     MapBuilder map = new MapBuilder("sessionid", sessionId);
     Response response = this.exec("get", SCHEDULER_INSTANCES_URL + "/" + instanceHandle , servLens, null, map);
     AssertUtil.assertSucceededResponse(response);
-    return response.readEntity(SchedulerJobInstanceInfo.class);
+    return response.readEntity(new GenericType<LensAPIResult<SchedulerJobInstanceInfo>>(){});
   }
 
-  public APIResult updateInstance(String instanceHandle, String action, String sessionId)
+  public LensAPIResult<Boolean> updateInstance(String instanceHandle, String action, String sessionId)
     throws JAXBException, IOException, ParseException, DatatypeConfigurationException {
 
     MapBuilder map = new MapBuilder("sessionid", sessionId, "action", action);
     Response response = this.exec("post", SCHEDULER_INSTANCES_URL + "/" + instanceHandle , servLens, null, map);
     AssertUtil.assertSucceededResponse(response);
-    return response.readEntity(APIResult.class);
+    return response.readEntity(new GenericType<LensAPIResult<Boolean>>(){});
   }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/main/java/org/apache/lens/regression/util/AssertUtil.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/main/java/org/apache/lens/regression/util/AssertUtil.java b/lens-regression/src/main/java/org/apache/lens/regression/util/AssertUtil.java
index b04e420..c865bf6 100644
--- a/lens-regression/src/main/java/org/apache/lens/regression/util/AssertUtil.java
+++ b/lens-regression/src/main/java/org/apache/lens/regression/util/AssertUtil.java
@@ -44,7 +44,6 @@ public class AssertUtil {
     Assert.assertEquals(response.getStatus(), Response.Status.OK.getStatusCode());
     APIResult result = response.readEntity(APIResult.class);
     Assert.assertEquals(result.getStatus(), APIResult.Status.SUCCEEDED);
-    Assert.assertNotNull(result.getMessage());
   }
 
   public static void assertSucceededResponse(Response response) {

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/main/resources/template.lens.properties
----------------------------------------------------------------------
diff --git a/lens-regression/src/main/resources/template.lens.properties b/lens-regression/src/main/resources/template.lens.properties
index 0df127d..a44ef28 100644
--- a/lens-regression/src/main/resources/template.lens.properties
+++ b/lens-regression/src/main/resources/template.lens.properties
@@ -18,8 +18,8 @@
 
 ######################
 
-lens.baseurl=
-lens.adminurl=
+lens.baseurl=http://localhost:9999/lensapi/
+lens.adminurl=http://localhost:9999/admin
 lens.username=
 lens.password=
 lens.server.dir=/usr/local/lens/server/
@@ -27,5 +27,8 @@ lens.client.dir=/usr/local/lens/client/
 lens.remote.host=
 lens.remote.username=
 lens.remote.password=
+lens.server.hdfsurl=hdfs://localhost:8020
+lens.server.currentDB=
+job.conf.url=http://localhost:19888/ws/v1/history/mapreduce/jobs/JOB_ID/conf
 
-######################
\ No newline at end of file
+######################

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/ITSessionTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/ITSessionTests.java b/lens-regression/src/test/java/org/apache/lens/regression/ITSessionTests.java
new file mode 100644
index 0000000..e106071
--- /dev/null
+++ b/lens-regression/src/test/java/org/apache/lens/regression/ITSessionTests.java
@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.lens.regression;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.Response;
+
+import javax.xml.bind.JAXBException;
+
+import org.apache.lens.api.query.LensQuery;
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+import org.apache.lens.regression.core.constants.DriverConfig;
+import org.apache.lens.regression.core.constants.QueryInventory;
+import org.apache.lens.regression.core.constants.SessionURL;
+import org.apache.lens.regression.core.helpers.*;
+import org.apache.lens.regression.core.testHelper.BaseTestClass;
+import org.apache.lens.regression.core.type.MapBuilder;
+import org.apache.lens.regression.util.Util;
+import org.apache.lens.server.api.LensConfConstants;
+import org.apache.lens.server.api.error.LensException;
+import org.apache.lens.server.api.util.LensUtil;
+
+import org.apache.log4j.Logger;
+
+import org.testng.Assert;
+import org.testng.annotations.*;
+
+import com.jcraft.jsch.JSchException;
+
+
+public class ITSessionTests extends BaseTestClass {
+
+  private WebTarget servLens;
+  private String sessionHandleString;
+
+  private static Logger logger = Logger.getLogger(ITSessionTests.class);
+
+  @BeforeClass(alwaysRun = true)
+  public void initialize() throws IOException, JSchException, JAXBException, LensException {
+    servLens = ServiceManagerHelper.init();
+  }
+
+  @BeforeMethod(alwaysRun = true)
+  public void setUp(Method method) throws Exception {
+    logger.info("Test Name: " + method.getName());
+    logger.info("Creating a new Session");
+    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+  }
+
+  @AfterMethod(alwaysRun = true)
+  public void closeSession() throws Exception {
+    logger.info("Closing Session");
+    sHelper.closeSession();
+  }
+
+
+  @Test(enabled = true)
+  public void testServerConfExposureInSession()  throws Exception {
+
+    // conf : lens-site.xml
+    MapBuilder query1 = new MapBuilder("sessionid", sessionHandleString, "key", LensConfConstants.SERVER_DB_JDBC_PASS);
+    Response response1 = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query1);
+    Assert.assertEquals(response1.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
+
+    //Driver conf : jdbc-driver.xml
+    MapBuilder query2 = new MapBuilder("sessionid", sessionHandleString, "key", "lens.driver.jdbc.db.user");
+    Response response2 = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query2);
+    Assert.assertEquals(response2.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
+  }
+
+
+  // LENS-760. Check for only running as queued is not fixed.
+  @Test(enabled = true)
+  public void testRunningQueryContinuationOnSessionClose()  throws Exception {
+
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "10");
+    String hiveDriverConf = lens.getServerDir() + "/conf/drivers/hive/hive1/hivedriver-site.xml";
+
+    try {
+      Util.changeConfig(map, hiveDriverConf);
+      lens.restart();
+
+      String session = sHelper.openSession("test", "test", lens.getCurrentDB());
+      List<QueryHandle> handleList = new ArrayList<QueryHandle>();
+      String sleepQuery = QueryInventory.getSleepQuery("5");
+
+      for(int i=1; i<=5; i++){
+        handleList.add((QueryHandle) qHelper.executeQuery(sleepQuery, null, session).getData());
+      }
+      qHelper.waitForQueryToRun(handleList.get(3));
+
+      List<QueryHandle> running = qHelper.getQueryHandleList(null, "RUNNING", "all", sessionHandleString);
+      sHelper.closeSession(session);
+      Assert.assertTrue(running.size() > 0);
+      logger.info("Running query count " + running.size());
+
+      for(QueryHandle handle : running){
+        LensQuery lq = qHelper.waitForCompletion(handle);
+        Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+      }
+    } finally {
+      Util.changeConfig(hiveDriverConf);
+      lens.restart();
+    }
+  }
+
+  // Fails. Bug : LENS-904
+  // Check for query continuation on session close.
+  @Test(enabled = true)
+  public void testQueryContinuationOnSessionClose()  throws Exception {
+
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "1");
+    String hiveDriverConf = lens.getServerDir() + "/conf/drivers/hive/hive1/hivedriver-site.xml";
+
+    try {
+      Util.changeConfig(map, hiveDriverConf);
+      lens.restart();
+
+      String session = sHelper.openSession("test", "test", lens.getCurrentDB());
+      List<QueryHandle> handleList = new ArrayList<QueryHandle>();
+      String sleepQuery = QueryInventory.getSleepQuery("3");
+
+      for (int i = 1; i <= 5; i++) {
+        handleList.add((QueryHandle) qHelper.executeQuery(sleepQuery, null, session).getData());
+      }
+
+      sHelper.closeSession(session);
+
+      for (QueryHandle handle : handleList) {
+        LensQuery lq = qHelper.waitForCompletion(handle);
+        Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+      }
+
+    } finally {
+      Util.changeConfig(hiveDriverConf);
+      lens.restart();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/SessionTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/SessionTests.java b/lens-regression/src/test/java/org/apache/lens/regression/SessionTests.java
deleted file mode 100644
index ce727a1..0000000
--- a/lens-regression/src/test/java/org/apache/lens/regression/SessionTests.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.lens.regression;
-
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import javax.ws.rs.client.WebTarget;
-import javax.ws.rs.core.Response;
-
-import javax.xml.bind.JAXBException;
-
-import org.apache.lens.api.query.LensQuery;
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryStatus;
-import org.apache.lens.regression.core.constants.DriverConfig;
-import org.apache.lens.regression.core.constants.QueryInventory;
-import org.apache.lens.regression.core.constants.SessionURL;
-import org.apache.lens.regression.core.helpers.*;
-import org.apache.lens.regression.core.testHelper.BaseTestClass;
-import org.apache.lens.regression.core.type.MapBuilder;
-import org.apache.lens.regression.util.Util;
-import org.apache.lens.server.api.LensConfConstants;
-import org.apache.lens.server.api.error.LensException;
-import org.apache.lens.server.api.util.LensUtil;
-
-import org.apache.log4j.Logger;
-
-import org.testng.Assert;
-import org.testng.annotations.*;
-
-import com.jcraft.jsch.JSchException;
-
-
-public class SessionTests extends BaseTestClass {
-
-  private WebTarget servLens;
-  private String sessionHandleString;
-
-  private static Logger logger = Logger.getLogger(SessionTests.class);
-
-  @BeforeClass(alwaysRun = true)
-  public void initialize() throws IOException, JSchException, JAXBException, LensException {
-    servLens = ServiceManagerHelper.init();
-  }
-
-  @BeforeMethod(alwaysRun = true)
-  public void setUp(Method method) throws Exception {
-    logger.info("Test Name: " + method.getName());
-    logger.info("Creating a new Session");
-    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
-  }
-
-  @AfterMethod(alwaysRun = true)
-  public void closeSession() throws Exception {
-    logger.info("Closing Session");
-    sHelper.closeSession();
-  }
-
-
-  @Test(enabled = true)
-  public void testServerConfExposureInSession()  throws Exception {
-
-    // conf : lens-site.xml
-    MapBuilder query1 = new MapBuilder("sessionid", sessionHandleString, "key", LensConfConstants.SERVER_DB_JDBC_PASS);
-    Response response1 = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query1);
-    Assert.assertEquals(response1.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
-
-    //Driver conf : jdbc-driver.xml
-    MapBuilder query2 = new MapBuilder("sessionid", sessionHandleString, "key", "lens.driver.jdbc.db.user");
-    Response response2 = lens.sendQuery("get", SessionURL.SESSION_PARAMS_URL, query2);
-    Assert.assertEquals(response2.getStatus(), Response.Status.NOT_FOUND.getStatusCode());
-  }
-
-
-  // LENS-760. Check for only running as queued is not fixed.
-  @Test(enabled = true)
-  public void testRunningQueryContinuationOnSessionClose()  throws Exception {
-
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "10");
-    String hiveDriverConf = lens.getServerDir() + "/conf/drivers/hive/hive1/hivedriver-site.xml";
-
-    try {
-      Util.changeConfig(map, hiveDriverConf);
-      lens.restart();
-
-      String session = sHelper.openSession("test", "test", lens.getCurrentDB());
-      List<QueryHandle> handleList = new ArrayList<QueryHandle>();
-      String sleepQuery = QueryInventory.getSleepQuery("5");
-
-      for(int i=1; i<=5; i++){
-        handleList.add((QueryHandle) qHelper.executeQuery(sleepQuery, null, session).getData());
-      }
-      qHelper.waitForQueryToRun(handleList.get(3));
-
-      List<QueryHandle> running = qHelper.getQueryHandleList(null, "RUNNING", "all", sessionHandleString);
-      sHelper.closeSession(session);
-      Assert.assertTrue(running.size() > 0);
-      logger.info("Running query count " + running.size());
-
-      for(QueryHandle handle : running){
-        LensQuery lq = qHelper.waitForCompletion(handle);
-        Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-      }
-    } finally {
-      Util.changeConfig(hiveDriverConf);
-      lens.restart();
-    }
-  }
-
-  // Fails. Bug : LENS-904
-  // Check for query continuation on session close.
-  @Test(enabled = true)
-  public void testQueryContinuationOnSessionClose()  throws Exception {
-
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "1");
-    String hiveDriverConf = lens.getServerDir() + "/conf/drivers/hive/hive1/hivedriver-site.xml";
-
-    try {
-      Util.changeConfig(map, hiveDriverConf);
-      lens.restart();
-
-      String session = sHelper.openSession("test", "test", lens.getCurrentDB());
-      List<QueryHandle> handleList = new ArrayList<QueryHandle>();
-      String sleepQuery = QueryInventory.getSleepQuery("3");
-
-      for (int i = 1; i <= 5; i++) {
-        handleList.add((QueryHandle) qHelper.executeQuery(sleepQuery, null, session).getData());
-      }
-
-      sHelper.closeSession(session);
-
-      for (QueryHandle handle : handleList) {
-        LensQuery lq = qHelper.waitForCompletion(handle);
-        Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-      }
-
-    } finally {
-      Util.changeConfig(hiveDriverConf);
-      lens.restart();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/client/ITDuplicateQueryTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/client/ITDuplicateQueryTests.java b/lens-regression/src/test/java/org/apache/lens/regression/client/ITDuplicateQueryTests.java
new file mode 100644
index 0000000..c2100da
--- /dev/null
+++ b/lens-regression/src/test/java/org/apache/lens/regression/client/ITDuplicateQueryTests.java
@@ -0,0 +1,188 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.lens.regression.client;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import javax.ws.rs.client.WebTarget;
+
+import org.apache.lens.api.LensConf;
+import org.apache.lens.api.query.*;
+import org.apache.lens.cube.parse.CubeQueryConfUtil;
+import org.apache.lens.regression.core.constants.DriverConfig;
+import org.apache.lens.regression.core.constants.QueryInventory;
+import org.apache.lens.regression.core.helpers.ServiceManagerHelper;
+import org.apache.lens.regression.core.testHelper.BaseTestClass;
+import org.apache.lens.regression.util.Util;
+import org.apache.lens.server.api.util.LensUtil;
+
+import org.apache.log4j.Logger;
+
+import org.testng.Assert;
+import org.testng.annotations.*;
+
+public class ITDuplicateQueryTests extends BaseTestClass {
+
+  WebTarget servLens;
+  private String sessionHandleString;
+
+  private String hiveDriverSitePath  = lens.getServerDir() + "/conf/drivers/hive/hive1/hivedriver-site.xml";
+  private static Logger logger = Logger.getLogger(ITDuplicateQueryTests.class);
+
+  @BeforeClass(alwaysRun = true)
+  public void initialize() throws Exception {
+    servLens = ServiceManagerHelper.init();
+  }
+
+  @BeforeMethod(alwaysRun = true)
+  public void setUp(Method method) throws Exception {
+    logger.info("Test Name: " + method.getName());
+    logger.info("Creating a new Session");
+    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+  }
+
+  @AfterMethod(alwaysRun = true)
+  public void closeSession() throws Exception {
+    logger.info("Closing Session");
+    sHelper.closeSession();
+  }
+
+  /* LENS-1019 : If query is repeated from user - with same query, same name, same conf on the same session
+     and earlier is still queued or running, then return the same handle.
+  */
+
+  @DataProvider(name = "query_names")
+  public Object[][] queryName() {
+    String[][] testData = {{"query-name"}, {null}};
+    return testData;
+  }
+
+  @Test(dataProvider = "query_names", enabled = true)
+  public void testRunningSameNameSessionQuery(String queryName) throws Exception {
+
+    String query = QueryInventory.getSleepQuery("10");
+    List<QueryHandle> handleList = new ArrayList<>();
+    List<PersistentQueryResult> resultList = new ArrayList<>();
+
+    for(int i=0; i<3; i++){
+      handleList.add((QueryHandle) qHelper.executeQuery(query, queryName).getData());
+    }
+
+    Assert.assertEquals(handleList.get(1).getHandleIdString(), handleList.get(0).getHandleIdString());
+    Assert.assertEquals(handleList.get(2).getHandleIdString(), handleList.get(0).getHandleIdString());
+
+    for(QueryHandle handle : handleList){
+      LensQuery lq = qHelper.waitForCompletion(handle);
+      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+      resultList.add((PersistentQueryResult) qHelper.getResultSet(handle));
+    }
+
+    Assert.assertEquals(resultList.get(1).getPersistedURI(), resultList.get(0).getPersistedURI());
+    Assert.assertEquals(resultList.get(2).getPersistedURI(), resultList.get(0).getPersistedURI());
+  }
+
+  @Test(enabled = true)
+  public void testQueuedSameNameSessionQuery() throws Exception {
+
+    String query = QueryInventory.getSleepQuery("10");
+    List<QueryHandle> handleList = new ArrayList<>();
+    List<PersistentQueryResult> resultList = new ArrayList<>();
+    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "1");
+
+    try {
+      Util.changeConfig(map, hiveDriverSitePath);
+      lens.restart();
+
+      //Fire long running query so that 2nd  query is in queued state
+      qHelper.executeQuery(query, "query1").getData();
+
+      for (int i = 0; i < 3; i++) {
+        handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_CUBE_QUERY, "query1").getData());
+      }
+
+      Assert.assertEquals(handleList.get(1), handleList.get(0));
+      Assert.assertEquals(handleList.get(2), handleList.get(0));
+
+      for (QueryHandle handle : handleList) {
+        LensQuery lq = qHelper.waitForCompletion(handle);
+        Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+        resultList.add((PersistentQueryResult) qHelper.getResultSet(handle));
+      }
+
+      Assert.assertEquals(resultList.get(1).getPersistedURI(), resultList.get(0).getPersistedURI());
+      Assert.assertEquals(resultList.get(2).getPersistedURI(), resultList.get(0).getPersistedURI());
+
+    } finally {
+      Util.changeConfig(hiveDriverSitePath);
+      lens.restart();
+    }
+  }
+
+  @Test(enabled = false)
+  public void differentQuerySameNameSession() throws Exception {
+
+    String cost5 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_5"), "5");
+    String cost3 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_3"), "3");
+
+    QueryHandle handle1 = (QueryHandle) qHelper.executeQuery(cost5, "queryName").getData();
+    QueryHandle handle2 = (QueryHandle) qHelper.executeQuery(cost3, "queryName").getData();
+
+    Assert.assertFalse(handle1.getHandleIdString().equals(handle2.getHandleIdString()));
+  }
+
+  @Test(enabled = false)
+  public void differentSessionSameNameQuery() throws Exception {
+
+    String query = QueryInventory.getSleepQuery("10");
+    String session1 = sHelper.openSession("user1", "pwd1", lens.getCurrentDB());
+    String session2 = sHelper.openSession("user2", "pwd2", lens.getCurrentDB());
+    QueryHandle handle1 = (QueryHandle) qHelper.executeQuery(query, "name", session1).getData();
+    QueryHandle handle2 = (QueryHandle) qHelper.executeQuery(query, "name", session2).getData();
+    Assert.assertFalse(handle1.getHandleIdString().equals(handle2.getHandleIdString()));
+  }
+
+  @Test(enabled = false)
+  public void differentNameSameSessionQuery() throws Exception {
+    String query = QueryInventory.getSleepQuery("3");
+    QueryHandle handle1 = (QueryHandle) qHelper.executeQuery(query, "name1").getData();
+    QueryHandle handle2 = (QueryHandle) qHelper.executeQuery(query, "name2").getData();
+    Assert.assertFalse(handle1.getHandleIdString().equals(handle2.getHandleIdString()));
+  }
+
+  @Test(enabled = false)
+  public void differentConfSameNameSessionQuery() throws Exception {
+
+    String query = QueryInventory.getSleepQuery("5");
+    LensConf lensConf = new LensConf();
+
+    lensConf.addProperty(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "false");
+    QueryHandle qhr1 = (QueryHandle) qHelper.executeQuery(query, "query-name", null, sessionHandleString,
+        lensConf).getData();
+
+    lensConf.addProperty(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "true");
+    QueryHandle qhr2 = (QueryHandle) qHelper.executeQuery(query, "query-name", null, sessionHandleString,
+         lensConf).getData();
+
+    Assert.assertFalse(qhr1.getHandleIdString().equals(qhr2.getHandleIdString()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/client/ITKillQueryTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/client/ITKillQueryTests.java b/lens-regression/src/test/java/org/apache/lens/regression/client/ITKillQueryTests.java
new file mode 100644
index 0000000..ecc0c6b
--- /dev/null
+++ b/lens-regression/src/test/java/org/apache/lens/regression/client/ITKillQueryTests.java
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.lens.regression.client;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Method;
+
+import javax.ws.rs.client.WebTarget;
+
+import javax.xml.bind.JAXBException;
+
+import org.apache.lens.api.query.*;
+import org.apache.lens.regression.core.constants.QueryInventory;
+import org.apache.lens.regression.core.helpers.*;
+import org.apache.lens.regression.core.testHelper.BaseTestClass;
+import org.apache.lens.server.api.error.LensException;
+
+import org.apache.log4j.Logger;
+
+import org.testng.Assert;
+import org.testng.annotations.*;
+
+
+public class ITKillQueryTests extends BaseTestClass {
+
+  WebTarget servLens;
+  private String sessionHandleString;
+
+  private final String hdfsJarPath = lens.getServerHdfsUrl() + "/tmp";
+  private final String localJarPath = new File("").getAbsolutePath() + "/lens-regression/target/testjars/";
+  private final String hiveUdfJar = "hiveudftest.jar";
+  private final String serverResourcePath = "/tmp/regression/resources";
+  String sleepQuery = QueryInventory.getSleepQuery("5");
+
+  private static Logger logger = Logger.getLogger(ITKillQueryTests.class);
+
+  @BeforeClass(alwaysRun = true)
+  public void initialize() throws IOException, JAXBException, LensException, IllegalAccessException,
+      InstantiationException {
+    servLens = ServiceManagerHelper.init();
+    logger.info("Creating a new Session");
+    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
+
+    //TODO : Enable when udf registration per driver is fixed
+/*  HadoopUtil.uploadJars(localJarPath + "/" + hiveUdfJar, hdfsJarPath);
+    logger.info("Adding jar for making query to run for longer period of time");
+    sHelper.addResourcesJar(hdfsJarPath + "/" + hiveUdfJar);
+    QueryHandle queryHandle = (QueryHandle) qHelper.executeQuery(QueryInventory.SLEEP_FUNCTION).getData();*/
+  }
+
+  @BeforeMethod(alwaysRun = true)
+  public void setUp(Method method) throws Exception {
+    logger.info("Test Name: " + method.getName());
+  }
+
+
+  @AfterClass(alwaysRun = true)
+  public void closeSession() throws Exception {
+    logger.info("Closing Session");
+    sHelper.closeSession();
+  }
+
+
+  @Test(enabled = true)
+  public void killQueryByHandle() throws Exception {
+
+    QueryHandle qH = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
+    logger.info("QUERY HANDLE : " + qH);
+
+    QueryStatus queryStatus = qHelper.waitForQueryToRun(qH);
+    Assert.assertEquals(queryStatus.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+
+    qHelper.killQueryByQueryHandle(qH);
+
+    queryStatus = qHelper.getQueryStatus(qH);
+    Assert.assertEquals(queryStatus.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+  }
+
+
+  @Test(enabled = true)
+  public void killQueryByUser() throws Exception {
+
+    String diffUser = "diff";
+    String diffPass = "diff";
+
+    String newSessionHandleSring = sHelper.openSession(diffUser, diffPass, lens.getCurrentDB());
+
+    logger.info("Adding jar for making query to run for longer period of time");
+    sHelper.addResourcesJar(hdfsJarPath + "/" + hiveUdfJar, newSessionHandleSring);
+
+    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
+    logger.info("1st QUERY HANDLE : " + queryHandle1);
+
+    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
+    logger.info("2nd QUERY HANDLE : " + queryHandle2);
+
+    QueryHandle queryHandle3 = (QueryHandle) qHelper.executeQuery(sleepQuery, null,
+        newSessionHandleSring).getData();
+    logger.info("3rd QUERY HANDLE : " + queryHandle3);
+
+    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
+    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
+    QueryStatus queryStatus3 = qHelper.waitForQueryToRun(queryHandle3, newSessionHandleSring);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+
+    Thread.sleep(1000);
+
+    qHelper.killQuery(null, null, lens.getUserName());
+    Thread.sleep(2000);
+    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
+    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
+    queryStatus3 = qHelper.getQueryStatus(queryHandle3);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+
+    qHelper.killQuery(null, null, diffUser);
+
+    queryStatus3 = qHelper.getQueryStatus(queryHandle3);
+    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+
+  }
+
+  @Test(enabled = true)
+  public void killQueryOfAllUser() throws Exception {
+
+    String diffUser = "diff";
+    String diffPass = "diff";
+    String newSessionHandleSring = sHelper.openSession(diffUser, diffPass, lens.getCurrentDB());
+    sHelper.addResourcesJar(hdfsJarPath + "/" + hiveUdfJar, newSessionHandleSring);
+
+    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
+    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
+    QueryHandle queryHandle3 = (QueryHandle) qHelper.executeQuery(sleepQuery, null,
+        newSessionHandleSring).getData();
+
+    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
+    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
+    QueryStatus queryStatus3 = qHelper.waitForQueryToRun(queryHandle3, newSessionHandleSring);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+
+    qHelper.killQuery(null, null, "all");
+
+    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
+    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
+    queryStatus3 = qHelper.getQueryStatus(queryHandle3);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+  }
+
+
+  @Test(enabled = true)
+  public void killAllQueryOfUser() throws Exception {
+
+    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
+    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
+
+    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
+    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+
+    qHelper.killQuery();
+
+    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
+    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+  }
+
+
+  @Test(enabled = true)
+  public void killQueryByState() throws Exception {
+
+    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
+    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
+
+    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
+    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+
+    //kill Running queries
+    qHelper.killQuery(null, "RUNNING");
+
+    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
+    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+
+    //kill Canceled query
+    qHelper.killQuery(null, "CANCELED");
+
+    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
+    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Running");
+
+    //kill successful query
+    QueryHandle queryHandle3 = (QueryHandle) qHelper.executeQuery(QueryInventory.QUERY).getData();
+    LensQuery lensQuery = qHelper.waitForCompletion(queryHandle3);
+    Assert.assertEquals(lensQuery.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
+    qHelper.killQuery(null, "SUCCESSFUL");
+    QueryStatus queryStatus3 = qHelper.getQueryStatus(queryHandle3);
+    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.SUCCESSFUL);
+  }
+
+  //TODO: enable when the bug is fixed.
+
+   /* Currently doing kill query by queryName "query" will kill all the query with queryName as "*query*"
+    * Raised a JIRA for same
+    * When its Fixed Revisit this function */
+
+  @Test(enabled = false)
+  public void killQueryByQueryName() throws Exception {
+
+    String queryName1 = "queryNameFirst";
+    String queryName2 = "queryNameSecond";
+    String queryName3 = "Name";
+
+    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery, queryName1).getData();
+    logger.info("1st QUERY HANDLE : " + queryHandle1);
+
+    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery, queryName2).getData();
+    logger.info("2nd QUERY HANDLE : " + queryHandle2);
+
+    QueryHandle queryHandle3 = (QueryHandle) qHelper.executeQuery(sleepQuery, queryName3).getData();
+    logger.info("3rd QUERY HANDLE : " + queryHandle3);
+
+    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
+    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
+    QueryStatus queryStatus3 = qHelper.waitForQueryToRun(queryHandle3);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+
+    qHelper.killQuery(queryName3);
+
+    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
+    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
+    queryStatus3 = qHelper.getQueryStatus(queryHandle3);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+    Assert.assertEquals(queryStatus3.getStatus(), QueryStatus.Status.CANCELED, "Query is Not Cancelled");
+
+    qHelper.killQuery(queryName1);
+
+    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
+    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not CANCELED");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+
+    qHelper.killQuery(queryName2);
+    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
+
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not CANCELED");
+  }
+
+  @Test(enabled = true)
+  public void killQueryByTimeRange() throws Exception {
+
+    String startTime1 = String.valueOf(System.currentTimeMillis());
+    logger.info("Start Time of 1st Query : " + startTime1);
+    Thread.sleep(1000);
+    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
+    Thread.sleep(1000);
+    String endTime1 = String.valueOf(System.currentTimeMillis());
+    logger.info("End Time of 1st Query : " + endTime1);
+
+    Thread.sleep(1000);
+
+    String startTime2 = String.valueOf(System.currentTimeMillis());
+    logger.info("Start Time of 2nd Query : " + startTime2);
+    Thread.sleep(1000);
+    QueryHandle queryHandle2 = (QueryHandle) qHelper.executeQuery(sleepQuery).getData();
+    Thread.sleep(1000);
+    String endTime2 = String.valueOf(System.currentTimeMillis());
+    logger.info("End Time of 2nd Query : " + endTime2);
+
+    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
+    QueryStatus queryStatus2 = qHelper.waitForQueryToRun(queryHandle2);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+
+    qHelper.killQuery(null, null, null, sessionHandleString, startTime1, endTime1);
+
+    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
+    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
+
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not CANCELED");
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.RUNNING, "Query is Not CANCELED");
+
+    qHelper.killQuery(null, null, null, sessionHandleString, startTime2, endTime2);
+
+    queryStatus2 = qHelper.getQueryStatus(queryHandle2);
+    Assert.assertEquals(queryStatus2.getStatus(), QueryStatus.Status.CANCELED, "Query is Not CANCELED");
+
+  }
+
+
+  @Test(enabled = true)
+  public void killQueryByAllFilter() throws Exception {
+
+    String queryName1 = "TestKill";
+
+    String startTime1 = String.valueOf(System.currentTimeMillis());
+    logger.info("Start Time of 1st Query : " + startTime1);
+    Thread.sleep(1000);
+
+    QueryHandle queryHandle1 = (QueryHandle) qHelper.executeQuery(sleepQuery, queryName1).getData();
+    Thread.sleep(1000);
+
+    String endTime1 = String.valueOf(System.currentTimeMillis());
+    logger.info("End Time of 1st Query : " + endTime1);
+
+    QueryStatus queryStatus1 = qHelper.waitForQueryToRun(queryHandle1);
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.RUNNING, "Query is Not Running");
+
+    qHelper.killQuery(queryName1, "RUNNING", lens.getUserName(), sessionHandleString, startTime1, endTime1);
+
+    queryStatus1 = qHelper.getQueryStatus(queryHandle1);
+    Assert.assertEquals(queryStatus1.getStatus(), QueryStatus.Status.CANCELED, "Query is Not CANCELED");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/client/ITListQueryTest.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/client/ITListQueryTest.java b/lens-regression/src/test/java/org/apache/lens/regression/client/ITListQueryTest.java
index cad2937..93de982 100644
--- a/lens-regression/src/test/java/org/apache/lens/regression/client/ITListQueryTest.java
+++ b/lens-regression/src/test/java/org/apache/lens/regression/client/ITListQueryTest.java
@@ -52,12 +52,12 @@ public class ITListQueryTest extends BaseTestClass {
   public void initialize() throws IOException, JAXBException, LensException {
     servLens = ServiceManagerHelper.init();
     logger.info("Creating a new Session");
-    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
   }
 
   @BeforeMethod(alwaysRun = true)
   public void setUp(Method method) throws Exception {
     logger.info("Test Name: " + method.getName());
+    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
   }
 
   @AfterMethod(alwaysRun = true)
@@ -65,12 +65,15 @@ public class ITListQueryTest extends BaseTestClass {
     logger.info("Test Name: " + method.getName());
     qHelper.killQuery(null, "QUEUED", "all");
     qHelper.killQuery(null, "RUNNING", "all");
+    if (sessionHandleString != null){
+      sHelper.closeSession();
+    }
+    sessionHandleString = null;
   }
 
   @AfterClass(alwaysRun = true)
   public void closeSession() throws Exception {
     logger.info("Closing Session");
-    sHelper.closeSession();
   }
 
   @DataProvider(name = "query-provider")

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/client/ITPreparedQueryTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/client/ITPreparedQueryTests.java b/lens-regression/src/test/java/org/apache/lens/regression/client/ITPreparedQueryTests.java
index fd73ba4..dc3d685 100644
--- a/lens-regression/src/test/java/org/apache/lens/regression/client/ITPreparedQueryTests.java
+++ b/lens-regression/src/test/java/org/apache/lens/regression/client/ITPreparedQueryTests.java
@@ -70,18 +70,27 @@ public class ITPreparedQueryTests extends BaseTestClass {
   public void initialize() throws IOException, JAXBException, LensException {
     servLens = ServiceManagerHelper.init();
     logger.info("Creating a new Session");
-    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
   }
 
   @BeforeMethod(alwaysRun = true)
   public void setUp(Method method) throws Exception {
+    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
     logger.info("Test Name: " + method.getName());
   }
 
+
+  @AfterMethod(alwaysRun = true)
+  public void afterMethod(Method method) throws Exception {
+    logger.info("Test Name: " + method.getName());
+    if (sessionHandleString != null){
+      sHelper.closeSession();
+    }
+    sessionHandleString = null;
+  }
+
   @AfterClass(alwaysRun = true)
   public void closeSession() throws Exception {
     logger.info("Closing Session");
-    sHelper.closeSession();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/client/ITQueryApiTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/client/ITQueryApiTests.java b/lens-regression/src/test/java/org/apache/lens/regression/client/ITQueryApiTests.java
index 7b8b377..cf6f82a 100644
--- a/lens-regression/src/test/java/org/apache/lens/regression/client/ITQueryApiTests.java
+++ b/lens-regression/src/test/java/org/apache/lens/regression/client/ITQueryApiTests.java
@@ -20,9 +20,7 @@
 package org.apache.lens.regression.client;
 
 import java.lang.reflect.Method;
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 
 import javax.ws.rs.client.WebTarget;
 import javax.ws.rs.core.GenericType;
@@ -33,7 +31,6 @@ import org.apache.lens.api.LensConf;
 import org.apache.lens.api.query.*;
 import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.cube.parse.CubeQueryConfUtil;
-import org.apache.lens.regression.core.constants.DriverConfig;
 import org.apache.lens.regression.core.constants.QueryInventory;
 import org.apache.lens.regression.core.constants.QueryURL;
 import org.apache.lens.regression.core.helpers.ServiceManagerHelper;
@@ -56,7 +53,7 @@ public class ITQueryApiTests extends BaseTestClass {
   WebTarget servLens;
   private String sessionHandleString;
 
-  private String hiveDriverSitePath  = lens.getServerDir() + "/conf/drivers/hive/hive1/hivedriver-site.xml";
+  String lensSiteConf = lens.getServerDir() + "/conf/lens-site.xml";
   private static Logger logger = Logger.getLogger(ITQueryApiTests.class);
 
   @BeforeClass(alwaysRun = true)
@@ -262,7 +259,6 @@ public class ITQueryApiTests extends BaseTestClass {
 //    InMemoryQueryResult ResultSetJson = (InMemoryQueryResult) qHelper.getResultSetJson(queryHandle1, "0", "100");
   }
 
-
   @Test
   public void testQueryResultJsonPersistent() throws Exception {
 
@@ -398,125 +394,6 @@ public class ITQueryApiTests extends BaseTestClass {
     Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
   }
 
-  /* LENS-1019 : If query is repeated from user - with same query, same name, same conf on the same session
-     and earlier is still queued or running, then return the same handle.
-  */
-
-  @DataProvider(name = "query_names")
-  public Object[][] queryName() {
-    String[][] testData = {{"query-name"}, {null}};
-    return testData;
-  }
-
-  @Test(dataProvider = "query_names", enabled = true)
-  public void testRunningSameNameSessionQuery(String queryName) throws Exception {
-
-    String query = QueryInventory.getSleepQuery("10");
-    List<QueryHandle> handleList = new ArrayList<>();
-    List<PersistentQueryResult> resultList = new ArrayList<>();
-
-    for(int i=0; i<3; i++){
-      handleList.add((QueryHandle) qHelper.executeQuery(query, queryName).getData());
-    }
-
-    Assert.assertEquals(handleList.get(1).getHandleIdString(), handleList.get(0).getHandleIdString());
-    Assert.assertEquals(handleList.get(2).getHandleIdString(), handleList.get(0).getHandleIdString());
-
-    for(QueryHandle handle : handleList){
-      LensQuery lq = qHelper.waitForCompletion(handle);
-      Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-      resultList.add((PersistentQueryResult) qHelper.getResultSet(handle));
-    }
-
-    Assert.assertEquals(resultList.get(1).getPersistedURI(), resultList.get(0).getPersistedURI());
-    Assert.assertEquals(resultList.get(2).getPersistedURI(), resultList.get(0).getPersistedURI());
-  }
-
-  @Test(enabled = true)
-  public void testQueuedSameNameSessionQuery() throws Exception {
-
-    String query = QueryInventory.getSleepQuery("10");
-    List<QueryHandle> handleList = new ArrayList<>();
-    List<PersistentQueryResult> resultList = new ArrayList<>();
-    HashMap<String, String> map = LensUtil.getHashMap(DriverConfig.MAX_CONCURRENT_QUERIES, "1");
-
-    try {
-      Util.changeConfig(map, hiveDriverSitePath);
-      lens.restart();
-
-      //Fire long running query so that 2nd  query is in queued state
-      qHelper.executeQuery(query, "query1").getData();
-
-      for (int i = 0; i < 3; i++) {
-        handleList.add((QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_CUBE_QUERY, "query1").getData());
-      }
-
-      Assert.assertEquals(handleList.get(1), handleList.get(0));
-      Assert.assertEquals(handleList.get(2), handleList.get(0));
-
-      for (QueryHandle handle : handleList) {
-        LensQuery lq = qHelper.waitForCompletion(handle);
-        Assert.assertEquals(lq.getStatus().getStatus(), QueryStatus.Status.SUCCESSFUL);
-        resultList.add((PersistentQueryResult) qHelper.getResultSet(handle));
-      }
-
-      Assert.assertEquals(resultList.get(1).getPersistedURI(), resultList.get(0).getPersistedURI());
-      Assert.assertEquals(resultList.get(2).getPersistedURI(), resultList.get(0).getPersistedURI());
-
-    } finally {
-      Util.changeConfig(hiveDriverSitePath);
-      lens.restart();
-    }
-  }
-
-  @Test(enabled = false)
-  public void differentQuerySameNameSession() throws Exception {
-
-    String cost5 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_5"), "5");
-    String cost3 = String.format(QueryInventory.getQueryFromInventory("HIVE.SLEEP_COST_3"), "3");
-
-    QueryHandle handle1 = (QueryHandle) qHelper.executeQuery(cost5, "queryName").getData();
-    QueryHandle handle2 = (QueryHandle) qHelper.executeQuery(cost3, "queryName").getData();
-
-    Assert.assertFalse(handle1.getHandleIdString().equals(handle2.getHandleIdString()));
-  }
-
-  @Test(enabled = false)
-  public void differentSessionSameNameQuery() throws Exception {
-
-    String query = QueryInventory.getSleepQuery("10");
-    String session1 = sHelper.openSession("user1", "pwd1", lens.getCurrentDB());
-    String session2 = sHelper.openSession("user2", "pwd2", lens.getCurrentDB());
-    QueryHandle handle1 = (QueryHandle) qHelper.executeQuery(query, "name", session1).getData();
-    QueryHandle handle2 = (QueryHandle) qHelper.executeQuery(query, "name", session2).getData();
-    Assert.assertFalse(handle1.getHandleIdString().equals(handle2.getHandleIdString()));
-  }
-
-  @Test(enabled = false)
-  public void differentNameSameSessionQuery() throws Exception {
-    String query = QueryInventory.getSleepQuery("3");
-    QueryHandle handle1 = (QueryHandle) qHelper.executeQuery(query, "name1").getData();
-    QueryHandle handle2 = (QueryHandle) qHelper.executeQuery(query, "name2").getData();
-    Assert.assertFalse(handle1.getHandleIdString().equals(handle2.getHandleIdString()));
-  }
-
-  @Test(enabled = false)
-  public void differentConfSameNameSessionQuery() throws Exception {
-
-    String query = QueryInventory.getSleepQuery("5");
-    LensConf lensConf = new LensConf();
-
-    lensConf.addProperty(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "false");
-    QueryHandle qhr1 = (QueryHandle) qHelper.executeQuery(query, "query-name", null, sessionHandleString,
-        lensConf).getData();
-
-    lensConf.addProperty(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "true");
-    QueryHandle qhr2 = (QueryHandle) qHelper.executeQuery(query, "query-name", null, sessionHandleString,
-         lensConf).getData();
-
-    Assert.assertFalse(qhr1.getHandleIdString().equals(qhr2.getHandleIdString()));
-  }
-
   // LENS-1186
   @Test
   public void testInvalidOperation() throws Exception {
@@ -556,4 +433,61 @@ public class ITQueryApiTests extends BaseTestClass {
     });
     Assert.assertEquals(result.getErrorCode(), 2005);
   }
+
+  //LENS-1304
+  @Test(enabled = true)
+  public void testDriverQueryAfterQueryPurge() throws Exception {
+
+    try{
+      HashMap<String, String> map = LensUtil.getHashMap(LensConfConstants.PURGE_INTERVAL, "3000");
+      Util.changeConfig(map, lensSiteConf);
+      lens.restart();
+
+      QueryHandle q1 = (QueryHandle) qHelper.executeQuery(QueryInventory.HIVE_DIM_QUERY).getData();
+      QueryHandle q2 = (QueryHandle) qHelper.executeQuery(QueryInventory.JDBC_DIM_QUERY).getData();
+
+      qHelper.waitForCompletion(q1);
+      qHelper.waitForCompletion(q2);
+
+      String hiveDriverQuery = "INSERT OVERWRITE DIRECTORY \""+lens.getServerHdfsUrl()+"/tmp/lensreports/hdfsout/"
+          + q1 + "\" ROW FORMAT SERDE 'org.apache.lens.lib.query.CSVSerde' STORED AS TEXTFILE SELECT "
+          + "(sample_dim2.id), (sample_dim2.name) FROM "+lens.getCurrentDB()+".local_dim_table2 sample_dim2 WHERE "
+          + "((((sample_dim2.name) != 'first') AND ((sample_dim2.dt = 'latest'))))";
+
+      String jdbcDriverQuery = "SELECT (sample_db_dim.id), (sample_db_dim.name) FROM " + lens.getCurrentDB()
+          + ".mydb_dim_table3 sample_db_dim WHERE ((((sample_db_dim.name) != 'first')))";
+
+      //Waiting for query to get purged
+      Thread.sleep(5000);
+
+      LensQuery l1 =  qHelper.getLensQuery(sessionHandleString, q1);
+      LensQuery l2 =  qHelper.getLensQuery(sessionHandleString, q2);
+
+      Assert.assertEquals(l1.getDriverQuery().trim(), hiveDriverQuery);
+      Assert.assertEquals(l2.getDriverQuery().trim(), jdbcDriverQuery);
+
+      //TODO : assert value from DB as well.
+
+    } catch(Exception e){
+      Util.changeConfig(lensSiteConf);
+      lens.restart();
+    }
+  }
+
+  @Test
+  public void testTimeout() throws Exception {
+
+    sHelper.setAndValidateParam(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "false");
+    sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_SET, "true");
+    sHelper.setAndValidateParam(LensConfConstants.QUERY_PERSISTENT_RESULT_INDRIVER, "false");
+    String query = QueryInventory.getSleepQuery("10");
+
+    Long t1 = System.currentTimeMillis();
+    QueryHandleWithResultSet qhr1 = (QueryHandleWithResultSet) qHelper.executeQueryTimeout(query, "80000",
+        null, sessionHandleString).getData();
+    Long t2 = System.currentTimeMillis();
+    long diff = (t2 - t1)/1000;
+    Assert.assertTrue(diff < 20); // adding 10 seconds extra buffer time
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/97fe26d3/lens-regression/src/test/java/org/apache/lens/regression/client/ITScheduleQueryTests.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/test/java/org/apache/lens/regression/client/ITScheduleQueryTests.java b/lens-regression/src/test/java/org/apache/lens/regression/client/ITScheduleQueryTests.java
deleted file mode 100644
index 4e45c4c..0000000
--- a/lens-regression/src/test/java/org/apache/lens/regression/client/ITScheduleQueryTests.java
+++ /dev/null
@@ -1,284 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.lens.regression.client;
-
-import java.lang.reflect.Method;
-import java.util.Calendar;
-import java.util.List;
-
-import javax.ws.rs.client.WebTarget;
-import javax.ws.rs.core.MediaType;
-
-import org.apache.lens.api.APIResult;
-import org.apache.lens.api.scheduler.*;
-import org.apache.lens.regression.core.constants.QueryInventory;
-import org.apache.lens.regression.core.helpers.ServiceManagerHelper;
-import org.apache.lens.regression.core.testHelper.BaseTestClass;
-import org.apache.lens.regression.util.Util;
-
-import org.apache.log4j.Logger;
-
-import org.testng.Assert;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-
-
-public class ITScheduleQueryTests extends BaseTestClass {
-
-  WebTarget servLens;
-  private String sessionHandleString;
-
-  private static Logger logger = Logger.getLogger(ITScheduleQueryTests.class);
-  private static String format = "yyyy-MM-dd HH:mm:ss";
-  private static String currentDate = Util.getCurrentDate(format);
-
-  @BeforeClass(alwaysRun = true)
-  public void initialize() throws Exception {
-    servLens = ServiceManagerHelper.init();
-  }
-
-  @BeforeMethod(alwaysRun = true)
-  public void setUp(Method method) throws Exception {
-    logger.info("Test Name: " + method.getName());
-    logger.info("Creating a new Session");
-    sessionHandleString = sHelper.openSession(lens.getCurrentDB());
-  }
-
-  @AfterMethod(alwaysRun = true)
-  public void closeSession() throws Exception {
-    logger.info("Closing Session");
-    if (sessionHandleString != null){
-      sHelper.closeSession();
-    }
-  }
-
-
-  @Test
-  public void submitJob() throws Exception {
-    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 4);
-    XJob xJob = scheduleHelper.getXJob("job-submit", QueryInventory.QUERY, null, currentDate, endDate,
-        XFrequencyEnum.DAILY);
-    String jobHandle = scheduleHelper.submitJob(xJob, sessionHandleString);
-    Assert.assertNotNull(jobHandle);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.NEW);
-  }
-
-  @Test
-  public void submitNScheduleQuery() throws Exception {
-
-    String startDate = Util.modifyDate(currentDate, format, Calendar.DATE, -1);
-    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 3);
-    XJob xJob = scheduleHelper.getXJob("job-submit-schedule", QueryInventory.JDBC_CUBE_QUERY, null, startDate,
-        endDate, XFrequencyEnum.DAILY);
-    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
-    Assert.assertNotNull(jobHandle);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SCHEDULED);
-
-    SchedulerJobInfo jobInfo = scheduleHelper.getJobDetails(jobHandle, sessionHandleString);
-    Assert.assertNotNull(jobInfo);
-    Assert.assertEquals(jobInfo.getJob().getName(), "job-submit-schedule");
-  }
-
-  //submit and schedule and also get job definition
-  @Test
-  public void submitNScheduleQueryCronExp() throws Exception {
-
-    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 1);
-    XJob xJob = scheduleHelper.getXJob("job-submit-schedule-cronExp", QueryInventory.QUERY, null, currentDate,
-        endDate, "0/30 * * * * ?");
-    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
-    Assert.assertNotNull(jobHandle);
-
-    XJob job = scheduleHelper.getJobDefinition(jobHandle, sessionHandleString, MediaType.APPLICATION_XML_TYPE,
-        MediaType.APPLICATION_XML);
-    Assert.assertNotNull(job);
-    Assert.assertEquals(job.getName(), "job-submit-schedule-cronExp");
-  }
-
-
-  @Test
-  public void testDeleteJob() throws Exception {
-
-    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 1);
-    XJob xJob = scheduleHelper.getXJob("job-delete", QueryInventory.QUERY, null, currentDate, endDate,
-         "0/30 * * * * ?");
-
-    //delete in submit state
-    String jobHandle = scheduleHelper.submitJob(xJob, sessionHandleString);
-    APIResult res = scheduleHelper.deleteJob(jobHandle, sessionHandleString);
-//    Assert.assertEquals(res.getStatus(), APIResult.Status.SUCCEEDED);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.DELETED);
-
-    //delete in scheduled state
-    jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
-    res = scheduleHelper.deleteJob(jobHandle, sessionHandleString);
-//    Assert.assertEquals(res.getStatus(), APIResult.Status.SUCCEEDED);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.DELETED);
-
-    //delete in suspended state
-    jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
-    scheduleHelper.updateJob(jobHandle, "SUSPEND", sessionHandleString);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SUSPENDED);
-    res = scheduleHelper.deleteJob(jobHandle, sessionHandleString);
-//    Assert.assertEquals(res.getStatus(), APIResult.Status.SUCCEEDED);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.DELETED);
-
-    //delete in expired state
-    jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
-    scheduleHelper.updateJob(jobHandle, "EXPIRE", sessionHandleString);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.EXPIRED);
-    res = scheduleHelper.deleteJob(jobHandle, sessionHandleString);
-//    Assert.assertEquals(res.getStatus(), APIResult.Status.SUCCEEDED);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.DELETED);
-  }
-
-  @Test
-  public void testUpdateJob() throws Exception {
-
-    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 4);
-    XJob job = scheduleHelper.getXJob("job-update", QueryInventory.QUERY, null, currentDate, endDate,
-        XFrequencyEnum.WEEKLY);
-    String jobHandle = scheduleHelper.submitJob(job, sessionHandleString);
-
-    XJob tmp = scheduleHelper.getJobDefinition(jobHandle, sessionHandleString);
-    tmp.setName("modified-name");
-    endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 6);
-    tmp.setEndTime(Util.getGregorianCalendar(endDate));
-    APIResult res = scheduleHelper.updateJob(tmp, jobHandle, sessionHandleString);
-    Assert.assertEquals(res.getStatus(), APIResult.Status.SUCCEEDED);
-
-    XJob modifiedJob = scheduleHelper.getJobDefinition(jobHandle, sessionHandleString);
-    Assert.assertEquals(modifiedJob.getName(), "modified-name");
-    String modifiedEndTime = Util.getDateStringFromGregorainCalender(modifiedJob.getEndTime(), format);
-    Assert.assertEquals(modifiedEndTime, endDate);
-  }
-
-  @Test
-  public void testUpdateJobAction() throws Exception {
-
-    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 1);
-    XJob job = scheduleHelper.getXJob("job-update-action", QueryInventory.QUERY, null, currentDate, endDate,
-        "0/20 * * * * ?");
-    String jobHandle = scheduleHelper.submitJob(job, sessionHandleString);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.NEW);
-
-    scheduleHelper.updateJob(jobHandle, "SCHEDULE", sessionHandleString);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SCHEDULED);
-
-    scheduleHelper.updateJob(jobHandle, "SUSPEND", sessionHandleString);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SUSPENDED);
-
-    scheduleHelper.updateJob(jobHandle, "RESUME", sessionHandleString);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SCHEDULED);
-
-    scheduleHelper.updateJob(jobHandle, "EXPIRE", sessionHandleString);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.EXPIRED);
-  }
-
-  @Test
-  public void testGetAllInstancesOfAJob() throws Exception {
-
-    String startDate = Util.modifyDate(Util.getCurrentDate(format), format, Calendar.SECOND, 5);
-    String endDate = Util.modifyDate(startDate, format, Calendar.MINUTE, 4);
-    XJob xJob = scheduleHelper.getXJob("job-update-action", QueryInventory.QUERY, null, startDate, endDate,
-        "0/20 * * * * ?");
-    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
-
-    Thread.sleep(60000);
-
-    List<SchedulerJobInstanceInfo> instanceList = scheduleHelper.getAllInstancesOfJob(jobHandle, "10",
-        sessionHandleString);
-    Assert.assertEquals(instanceList.size(), 3);
-  }
-
-
-  @Test
-  public void updateInstance() throws Exception {
-
-    String startDate = Util.modifyDate(Util.getCurrentDate(format), format, Calendar.SECOND, 10);
-    String endDate = Util.modifyDate(startDate, format, Calendar.MINUTE, 3);
-    XJob xJob = scheduleHelper.getXJob("job-update-action", QueryInventory.JDBC_DIM_QUERY, null, startDate, endDate,
-        "0/20 * * * * ?");
-    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
-
-    Thread.sleep(20000);
-
-    List<SchedulerJobInstanceInfo> instanceList = scheduleHelper.getAllInstancesOfJob(jobHandle, "10",
-        sessionHandleString);
-
-    Thread.sleep(10000);
-
-    APIResult res = scheduleHelper.updateInstance(instanceList.get(0).getId().getHandleIdString(),
-        "RERUN", sessionHandleString);
-    Assert.assertEquals(res.getStatus(), APIResult.Status.SUCCEEDED);
-
-    SchedulerJobInstanceInfo instanceInfo = scheduleHelper.getInstanceDetails(instanceList.get(0).getId()
-        .getHandleIdString(), sessionHandleString);
-    List<SchedulerJobInstanceRun> runList = instanceInfo.getInstanceRunList();
-    Assert.assertEquals(runList.size(), 2);
-    Assert.assertEquals(runList.get(1).getRunId(), 2);
-  }
-
-
-  @Test(enabled = true)
-  public void restart() throws Exception {
-
-    String startDate = Util.modifyDate(Util.getCurrentDate(format), format, Calendar.SECOND, 5);
-    String endDate = Util.modifyDate(startDate, format, Calendar.MINUTE, 2);
-    XJob xJob = scheduleHelper.getXJob("job-restart", QueryInventory.QUERY, null, startDate, endDate, "0/20 * * * * ?");
-
-    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
-    Assert.assertNotNull(jobHandle);
-
-    Thread.sleep(20000);
-    lens.stop();
-    Thread.sleep(20000);
-    lens.start();
-    Thread.sleep(60000);
-
-    List<SchedulerJobInstanceInfo> instanceList = scheduleHelper.getAllInstancesOfJob(jobHandle, "50",
-        sessionHandleString);
-    Assert.assertEquals(instanceList.size(), 6);
-  }
-
-  //LENS
-  @Test
-  public void testMisfiredEvents() throws Exception {
-
-    String startDate = Util.modifyDate(currentDate, format, Calendar.DATE, -2);
-    String endDate = Util.modifyDate(currentDate, format, Calendar.DATE, 3);
-    XJob xJob = scheduleHelper.getXJob("job-misfire", QueryInventory.JDBC_CUBE_QUERY, null, startDate,
-        endDate, XFrequencyEnum.DAILY);
-    String jobHandle = scheduleHelper.submitNScheduleJob(xJob, sessionHandleString);
-    Assert.assertNotNull(jobHandle);
-    Assert.assertEquals(scheduleHelper.getJobStatus(jobHandle), SchedulerJobState.SCHEDULED);
-
-    List<SchedulerJobInstanceInfo> instanceList = scheduleHelper.getAllInstancesOfJob(jobHandle, "10",
-        sessionHandleString);
-    Assert.assertEquals(instanceList.size(), 3);
-
-    if (jobHandle!=null){
-      scheduleHelper.updateJob(jobHandle, "EXPIRE", sessionHandleString);
-    }
-  }
-}


[11/16] lens git commit: LENS-1386 : Add support for separate tables for update periods in one storage

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/lens/blob/f0dadd79/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java b/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
index 0e6a4a1..f6f6e77 100644
--- a/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
+++ b/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
@@ -62,7 +62,6 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 
 import org.glassfish.jersey.test.TestProperties;
-
 import org.testng.Assert;
 import org.testng.annotations.*;
 
@@ -974,7 +973,29 @@ public class TestMetastoreService extends LensJerseyTest {
     final String[] timePartColNames = {"dt"};
     return createStorageTblElement(storageName, table, timePartColNames, updatePeriod);
   }
+  private XStorageTableElement createStorageTblWithMultipleTableDescriptors(String storageName, String[] tables,
+     String [] updatePeriods) {
+    String [][] timePartColNames  = new String[updatePeriods.length][];
+    for (int i = 0; i < updatePeriods.length; i++) {
+      timePartColNames[i] = new String[]{ "dt" };
+    }
+    return createStorageTblWithMultipleTableDescriptors(storageName, tables, timePartColNames, updatePeriods);
+  }
 
+  private XStorageTableElement createStorageTblWithMultipleTableDescriptors(String storageName, String[] tables,
+    String[][] timePartColNames, String [] updatePeriods) {
+    XStorageTableElement tbl = cubeObjectFactory.createXStorageTableElement();
+    tbl.setStorageName(storageName);
+    XUpdatePeriods xUpdatePeriods = new XUpdatePeriods();
+    tbl.setUpdatePeriods(xUpdatePeriods);
+    for (int i = 0; i < updatePeriods.length; i++) {
+      XUpdatePeriodTableDescriptor updatePeriodTableDescriptor = new XUpdatePeriodTableDescriptor();
+      updatePeriodTableDescriptor.setUpdatePeriod(XUpdatePeriod.valueOf(updatePeriods[i]));
+      updatePeriodTableDescriptor.setTableDesc(createStorageTableDesc(tables[i], timePartColNames[i]));
+      xUpdatePeriods.getUpdatePeriodTableDescriptor().add(updatePeriodTableDescriptor);
+    }
+    return tbl;
+  }
   private XStorageTableElement createStorageTblElement(String storageName, String table,
     final String[] timePartColNames, String... updatePeriod) {
     XStorageTableElement tbl = cubeObjectFactory.createXStorageTableElement();
@@ -1836,6 +1857,169 @@ public class TestMetastoreService extends LensJerseyTest {
   }
 
   @Test(dataProvider = "mediaTypeData")
+  public void testCreateFactTableWithMultipleUpdatePeriods(MediaType mediaType) throws Exception {
+
+    final String table = "testCreateFactTableWithMultipleUpdatePeriods";
+    String prevDb = getCurrentDatabase(mediaType);
+    final String DB = dbPFX + "testCreateFactTableWithMultipleUpdatePeriods_DB" + mediaType.getSubtype();
+    createDatabase(DB, mediaType);
+    setCurrentDatabase(DB, mediaType);
+    createStorage("S1", mediaType);
+    try {
+      final XCube cube = createTestCube("testCube");
+      target().path("metastore").path("cubes").queryParam("sessionid", lensSessionId).request(mediaType)
+        .post(Entity.entity(new GenericEntity<JAXBElement<XCube>>(cubeObjectFactory.createXCube(cube)) {
+        }, mediaType), APIResult.class);
+      XFactTable f = createFactTable(table);
+      String[] tables = new String[] { "testTable1", "testTable2", "testTable3" };
+      String[] updatePeriods = new String[] { "HOURLY", "DAILY", "MONTHLY" };
+      f.getStorageTables().getStorageTable()
+        .add(createStorageTblWithMultipleTableDescriptors("S1", tables, updatePeriods));
+      APIResult result = target().path("metastore").path("facts").queryParam("sessionid", lensSessionId)
+        .request(mediaType)
+        .post(Entity.entity(new GenericEntity<JAXBElement<XFactTable>>(cubeObjectFactory.createXFactTable(f)) {
+        }, mediaType), APIResult.class);
+      assertSuccess(result);
+
+      StringList factNames = target().path("metastore/facts").queryParam("sessionid", lensSessionId).request(mediaType)
+        .get(StringList.class);
+      assertTrue(factNames.getElements().contains(table.toLowerCase()));
+
+      // Get the created tables
+      JAXBElement<XFactTable> gotFactElement = target().path("metastore/facts").path(table)
+        .queryParam("sessionid", lensSessionId).request(mediaType).get(new GenericType<JAXBElement<XFactTable>>() {
+        });
+      XFactTable gotFact = gotFactElement.getValue();
+      assertTrue(gotFact.getName().equalsIgnoreCase(table));
+      assertEquals(gotFact.getWeight(), 10.0);
+
+      // Check for the created tables per update period.
+      List<XUpdatePeriodTableDescriptor> updatePeriodTableDescriptor = gotFact.getStorageTables().getStorageTable()
+        .get(0).getUpdatePeriods().getUpdatePeriodTableDescriptor();
+      assertEquals(updatePeriodTableDescriptor.size(), 3);
+
+      CubeFactTable cf = JAXBUtils.cubeFactFromFactTable(gotFact);
+
+      Map<UpdatePeriod, String> updatePeriodTablePrefixMap = cf.getStoragePrefixUpdatePeriodMap().get("S1");
+      for (Map.Entry entry : updatePeriodTablePrefixMap.entrySet()) {
+        assertEquals(entry.getValue(), entry.getKey() + "_S1");
+      }
+      // Do some changes to test update
+      cf.alterWeight(20.0);
+      cf.alterColumn(new FieldSchema("c2", "double", "changed to double"));
+
+      XFactTable update = JAXBUtils.factTableFromCubeFactTable(cf);
+      XStorageTableElement s1Tbl = createStorageTblWithMultipleTableDescriptors("S1",
+        new String[] { tables[0], tables[1] }, new String[] { updatePeriods[0], updatePeriods[1] });
+      update.getStorageTables().getStorageTable().add(s1Tbl);
+      // Update
+      result = target().path("metastore").path("facts").path(table).queryParam("sessionid", lensSessionId)
+        .request(mediaType)
+        .put(Entity.entity(new GenericEntity<JAXBElement<XFactTable>>(cubeObjectFactory.createXFactTable(update)) {
+        }, mediaType), APIResult.class);
+      assertSuccess(result);
+
+      // Get the updated table
+      gotFactElement = target().path("metastore/facts").path(table).queryParam("sessionid", lensSessionId)
+        .request(mediaType).get(new GenericType<JAXBElement<XFactTable>>() {
+        });
+      gotFact = gotFactElement.getValue();
+      CubeFactTable ucf = JAXBUtils.cubeFactFromFactTable(gotFact);
+      assertEquals(ucf.weight(), 20.0);
+      assertTrue(ucf.getUpdatePeriods().get("S1").contains(HOURLY));
+      assertTrue(ucf.getUpdatePeriods().get("S1").contains(DAILY));
+      assertFalse(ucf.getUpdatePeriods().get("S1").contains(MONTHLY));
+
+      // Add partitions
+      final Date partDate = new Date();
+      XPartition xp = createPartition(table, partDate);
+      APIResult partAddResult = target().path("metastore/facts/").path(table).path("storages/S1/partition")
+        .queryParam("sessionid", lensSessionId).request(mediaType)
+        .post(Entity.entity(new GenericEntity<JAXBElement<XPartition>>(cubeObjectFactory.createXPartition(xp)) {
+        }, mediaType), APIResult.class);
+      assertSuccess(partAddResult);
+
+      // add same should fail
+      partAddResult = target().path("metastore/facts/").path(table).path("storages/S1/partition")
+        .queryParam("sessionid", lensSessionId).request(mediaType)
+        .post(Entity.entity(new GenericEntity<JAXBElement<XPartition>>(cubeObjectFactory.createXPartition(xp)) {
+        }, mediaType), APIResult.class);
+      assertEquals(partAddResult.getStatus(), Status.FAILED);
+
+      xp.setLocation(xp.getLocation() + "/a/b/c");
+      APIResult partUpdateResult = target().path("metastore/facts/").path(table).path("storages/S1/partition")
+        .queryParam("sessionid", lensSessionId).request(mediaType)
+        .put(Entity.entity(new GenericEntity<JAXBElement<XPartition>>(cubeObjectFactory.createXPartition(xp)) {
+        }, mediaType), APIResult.class);
+      assertSuccess(partUpdateResult);
+
+      JAXBElement<XPartitionList> partitionsElement = target().path("metastore/facts").path(table)
+        .path("storages/S1/partitions").queryParam("sessionid", lensSessionId).request(mediaType)
+        .get(new GenericType<JAXBElement<XPartitionList>>() {
+        });
+
+      XPartitionList partitions = partitionsElement.getValue();
+      assertNotNull(partitions);
+      assertEquals(partitions.getPartition().size(), 1);
+      XPartition readPartition = partitions.getPartition().get(0);
+      assertEquals(readPartition.getLocation(), xp.getLocation());
+      assertEquals(readPartition.getTimePartitionSpec(), xp.getTimePartitionSpec());
+      assertEquals(readPartition.getNonTimePartitionSpec(), xp.getNonTimePartitionSpec());
+      assertNotNull(readPartition.getFullPartitionSpec());
+      XTimePartSpecElement timePartSpec = readPartition.getTimePartitionSpec().getPartSpecElement().iterator().next();
+      XPartSpecElement fullPartSpec = readPartition.getFullPartitionSpec().getPartSpecElement().iterator().next();
+      assertEquals(timePartSpec.getKey(), fullPartSpec.getKey());
+      assertEquals(
+        UpdatePeriod.valueOf(xp.getUpdatePeriod().name()).format(JAXBUtils.getDateFromXML(timePartSpec.getValue())),
+        fullPartSpec.getValue());
+      DateTime date = target().path("metastore/cubes").path("testCube").path("latestdate")
+        .queryParam("timeDimension", "dt").queryParam("sessionid", lensSessionId).request(mediaType)
+        .get(DateTime.class);
+
+      partDate.setMinutes(0);
+      partDate.setSeconds(0);
+      partDate.setTime(partDate.getTime() - partDate.getTime() % 1000);
+      assertEquals(date.getDate(), partDate);
+      // add two partitions, one of them already added. result should be partial
+      XPartitionList parts = new XPartitionList();
+      parts.getPartition().add(xp);
+      parts.getPartition().add(createPartition(table, DateUtils.addHours(partDate, 1)));
+      partAddResult = target().path("metastore/facts/").path(table).path("storages/S1/partitions")
+        .queryParam("sessionid", lensSessionId).request(mediaType).post(
+          Entity.entity(new GenericEntity<JAXBElement<XPartitionList>>(cubeObjectFactory.createXPartitionList(parts)) {
+          }, mediaType), APIResult.class);
+      assertEquals(partAddResult.getStatus(), Status.PARTIAL);
+
+      // Drop the partitions
+      APIResult dropResult = target().path("metastore/facts").path(table).path("storages/S1/partitions")
+        .queryParam("sessionid", lensSessionId).request(mediaType).delete(APIResult.class);
+
+      assertSuccess(dropResult);
+
+      // Verify partition was dropped
+      partitionsElement = target().path("metastore/facts").path(table).path("storages/S1/partitions")
+        .queryParam("sessionid", lensSessionId).request(mediaType).get(new GenericType<JAXBElement<XPartitionList>>() {
+        });
+
+      partitions = partitionsElement.getValue();
+      assertNotNull(partitions);
+      assertEquals(partitions.getPartition().size(), 0);
+      // add null in batch
+      Response resp = target().path("metastore/facts/").path(table).path("storages/S1/partitions")
+        .queryParam("sessionid", lensSessionId).request(mediaType).post(null);
+      Assert.assertEquals(resp.getStatus(), 400);
+
+      // Drop the cube
+      WebTarget target = target().path("metastore").path("cubes").path("testCube");
+      result = target.queryParam("sessionid", lensSessionId).request(mediaType).delete(APIResult.class);
+      assertSuccess(result);
+    } finally {
+      setCurrentDatabase(prevDb, mediaType);
+      dropDatabase(DB, mediaType);
+    }
+  }
+
+  @Test(dataProvider = "mediaTypeData")
   public void testCreateFactTable(MediaType mediaType) throws Exception {
     final String table = "testCreateFactTable";
     final String DB = dbPFX + "testCreateFactTable_DB" + mediaType.getSubtype();


[10/16] lens git commit: LENS-1387: Move Lens compile & target version to Java 8

Posted by pr...@apache.org.
LENS-1387: Move Lens compile & target version to Java 8


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

Branch: refs/heads/lens-1381
Commit: 0cd22b107ba311220a3038827f6977aa460cdcfe
Parents: 97fe26d
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Thu Feb 16 08:54:46 2017 +0800
Committer: raju <ra...@apache.org>
Committed: Thu Feb 16 08:54:46 2017 +0800

----------------------------------------------------------------------
 pom.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/0cd22b10/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8ea64b7..12cd9cc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -34,8 +34,8 @@
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
 
     <!-- java and javax -->
-    <java.source.version>1.7</java.source.version>
-    <java.target.version>1.7</java.target.version>
+    <java.source.version>1.8</java.source.version>
+    <java.target.version>1.8</java.target.version>
     <servlet3.version>3.0.1</servlet3.version>
     <servlet.api.version>2.5</servlet.api.version>
     <ws.rs.version>2.0.1</ws.rs.version>


[15/16] lens git commit: LENS-1398(incremental): Python client not raising exception in some cases

Posted by pr...@apache.org.
LENS-1398(incremental): Python client not raising exception in some cases


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

Branch: refs/heads/lens-1381
Commit: d1b43d615ce950609d0496ad8db1a63813f3042b
Parents: a3045fe
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Mon Mar 27 14:42:38 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Mon Mar 27 14:42:38 2017 +0530

----------------------------------------------------------------------
 contrib/clients/python/lens/client/main.py | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/d1b43d61/contrib/clients/python/lens/client/main.py
----------------------------------------------------------------------
diff --git a/contrib/clients/python/lens/client/main.py b/contrib/clients/python/lens/client/main.py
index 1e9d88f..b1846ad 100644
--- a/contrib/clients/python/lens/client/main.py
+++ b/contrib/clients/python/lens/client/main.py
@@ -25,7 +25,8 @@ import logging
 logger = logging.getLogger(__name__)
 
 class LensClient(object):
-    def __init__(self, base_url=None, username="", password="", database=None, conf=None):
+    def __init__(self, base_url=None, username="", password="", database=None, conf=None, logging_level=logging.INFO):
+        logging.basicConfig(level=logging_level)
         if conf and isinstance(conf, string_types) and os.path.exists(conf):
             if os.path.isdir(conf):
                 conf = os.path.join(conf, 'lens-client-site.xml')


[06/16] lens git commit: LENS-773: Include dimension table partitions in FactPartitionBasedQueryCostCalculator

Posted by pr...@apache.org.
LENS-773: Include dimension table partitions in FactPartitionBasedQueryCostCalculator


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

Branch: refs/heads/lens-1381
Commit: 6af57fb6748eaf2501258d1f9756c3476a6e3087
Parents: d46e78c
Author: Amareshwari Sriramadasu <am...@apache.org>
Authored: Mon Jan 30 12:42:14 2017 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Mon Jan 30 12:42:14 2017 +0530

----------------------------------------------------------------------
 .../FactPartitionBasedQueryCostCalculator.java  |  3 +++
 ...stFactPartitionBasedQueryCostCalculator.java | 21 +++++++++++++++-----
 2 files changed, 19 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/6af57fb6/lens-cube/src/main/java/org/apache/lens/cube/query/cost/FactPartitionBasedQueryCostCalculator.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/query/cost/FactPartitionBasedQueryCostCalculator.java b/lens-cube/src/main/java/org/apache/lens/cube/query/cost/FactPartitionBasedQueryCostCalculator.java
index 9fecdbc..3c157ee 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/query/cost/FactPartitionBasedQueryCostCalculator.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/query/cost/FactPartitionBasedQueryCostCalculator.java
@@ -67,6 +67,9 @@ public class FactPartitionBasedQueryCostCalculator implements QueryCostCalculato
           }
           cost += allTableWeights * getNormalizedUpdatePeriodCost(partition.getPeriod(), driver);
         }
+      } else {
+        // increase cost for every dimtable partition
+        cost += 1.0;
       }
     }
     return cost;

http://git-wip-us.apache.org/repos/asf/lens/blob/6af57fb6/lens-cube/src/test/java/org/apache/lens/cube/query/cost/TestFactPartitionBasedQueryCostCalculator.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/query/cost/TestFactPartitionBasedQueryCostCalculator.java b/lens-cube/src/test/java/org/apache/lens/cube/query/cost/TestFactPartitionBasedQueryCostCalculator.java
index 262d452..b5e73fd 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/query/cost/TestFactPartitionBasedQueryCostCalculator.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/query/cost/TestFactPartitionBasedQueryCostCalculator.java
@@ -25,6 +25,7 @@ import static org.apache.lens.cube.metadata.UpdatePeriod.*;
 
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
+import static org.testng.Assert.*;
 
 import java.util.HashMap;
 import java.util.Set;
@@ -43,12 +44,12 @@ import org.testng.annotations.Test;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
-import junit.framework.Assert;
 
 public class TestFactPartitionBasedQueryCostCalculator {
   AbstractQueryContext queryContext;
   FactPartitionBasedQueryCostCalculator calculator = new FactPartitionBasedQueryCostCalculator();
   LensDriver driver;
+  private static String latest = "latest";
 
   @BeforeTest
   public void setUp() {
@@ -68,6 +69,7 @@ public class TestFactPartitionBasedQueryCostCalculator {
     HashMap<String, Set<?>> partitions = new HashMap<>();
     partitions.put("st1", Sets.newHashSet(fp1, fp2));
     partitions.put("st2", Sets.newHashSet(fp3, fp4));
+    partitions.put("st3", Sets.newHashSet(latest));
     DriverQueryPlan plan = mock(DriverQueryPlan.class);
     when(queryContext.getDriverRewriterPlan(driver)).thenReturn(plan);
     when(plan.getPartitions()).thenReturn(partitions);
@@ -85,12 +87,21 @@ public class TestFactPartitionBasedQueryCostCalculator {
   @Test
   public void testCalculateCost() throws Exception {
     QueryCost cost = calculator.calculateCost(queryContext, driver);
-    Assert.assertTrue(cost.getEstimatedResourceUsage() > 18.0);
-    Assert.assertTrue(cost.getEstimatedResourceUsage() < 19.0);
+    assertTrue(cost.getEstimatedResourceUsage() > 19.0, "Estimated resource usage:" + cost.getEstimatedResourceUsage());
+    assertTrue(cost.getEstimatedResourceUsage() < 20.0, "Estimated resource usage:" + cost.getEstimatedResourceUsage());
   }
 
   @Test
-  public void testGetAllPartitions() throws Exception {
-
+  public void testDimensionCost() throws Exception {
+    AbstractQueryContext queryContext2 = mock(AbstractQueryContext.class);
+    HashMap<String, Set<?>> partitions = new HashMap<>();
+    partitions.put("st1", Sets.newHashSet(latest));
+    partitions.put("st2", Sets.newHashSet(latest));
+    DriverQueryPlan plan = mock(DriverQueryPlan.class);
+    when(queryContext2.getDriverRewriterPlan(driver)).thenReturn(plan);
+    when(plan.getPartitions()).thenReturn(partitions);
+    when(calculator.getAllPartitions(queryContext2, driver)).thenReturn(partitions);
+    QueryCost cost = calculator.calculateCost(queryContext2, driver);
+    assertTrue(cost.getEstimatedResourceUsage() == 2.0, "Estimated resource usage:" + cost.getEstimatedResourceUsage());
   }
 }


[12/16] lens git commit: LENS-1386 : Add support for separate tables for update periods in one storage

Posted by pr...@apache.org.
LENS-1386 : Add support for separate tables for update periods in one storage


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

Branch: refs/heads/lens-1381
Commit: f0dadd79bb626fe6f8bbf21569e3062aeb9be070
Parents: 0cd22b1
Author: Lavkesh Lahngir <la...@linux.com>
Authored: Mon Feb 20 15:08:40 2017 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Mon Feb 20 15:08:40 2017 +0530

----------------------------------------------------------------------
 lens-api/src/main/resources/cube-0.1.xsd        |  28 +-
 .../lens/cube/metadata/CubeFactTable.java       |  68 +++-
 .../lens/cube/metadata/CubeMetastoreClient.java | 339 +++++++++++--------
 .../lens/cube/metadata/MetastoreUtil.java       |   6 +
 .../org/apache/lens/cube/metadata/Storage.java  |  30 +-
 .../cube/metadata/TestCubeMetastoreClient.java  | 151 ++++++++-
 .../metastore/CubeMetastoreServiceImpl.java     | 182 ++++++----
 .../apache/lens/server/metastore/JAXBUtils.java |  66 +++-
 .../server/metastore/TestMetastoreService.java  | 186 +++++++++-
 9 files changed, 811 insertions(+), 245 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/f0dadd79/lens-api/src/main/resources/cube-0.1.xsd
----------------------------------------------------------------------
diff --git a/lens-api/src/main/resources/cube-0.1.xsd b/lens-api/src/main/resources/cube-0.1.xsd
index f438f48..431d68b 100644
--- a/lens-api/src/main/resources/cube-0.1.xsd
+++ b/lens-api/src/main/resources/cube-0.1.xsd
@@ -681,8 +681,27 @@
   </xs:complexType>
 
   <xs:complexType name="x_update_periods">
-    <xs:sequence>
+    <xs:annotation>
+      <xs:documentation>
+        A list of update_period which contains either update period table descriptor or list of update_peroid enum.
+      </xs:documentation>
+    </xs:annotation>
+    <xs:choice maxOccurs="1" minOccurs="0">
+      <xs:element name="update_period_table_descriptor" type="x_update_period_table_descriptor" maxOccurs="unbounded"
+                  minOccurs="0"/>
       <xs:element name="update_period" type="x_update_period" maxOccurs="unbounded" minOccurs="0"/>
+    </xs:choice>
+  </xs:complexType>
+
+  <xs:complexType name="x_update_period_table_descriptor">
+    <xs:annotation>
+      <xs:documentation>
+        An update period descriptor keeps an enum of update period and a storage table descriptor.
+      </xs:documentation>
+    </xs:annotation>
+    <xs:sequence>
+      <xs:element name="update_period" type="x_update_period" maxOccurs="1" minOccurs="1"/>
+      <xs:element name="table_desc" type="x_storage_table_desc" maxOccurs="1" minOccurs="1"/>
     </xs:sequence>
   </xs:complexType>
 
@@ -1001,13 +1020,14 @@
   <xs:complexType name="x_storage_table_element">
     <xs:annotation>
       <xs:documentation>
-        Storage and storage table description and update periods
+        Storage and storage table description and update periods. table_desc is invalid when update_periods has a list
+        of update_period_table_descriptor instead of a list of enums.
       </xs:documentation>
     </xs:annotation>
     <xs:sequence>
-      <xs:element name="update_periods" type="x_update_periods" maxOccurs="1" minOccurs="0"/>
+      <xs:element name="update_periods" type="x_update_periods" maxOccurs="1" minOccurs="1"/>
       <xs:element name="storage_name" type="xs:string"/>
-      <xs:element type="x_storage_table_desc" name="table_desc"/>
+      <xs:element type="x_storage_table_desc" name="table_desc" maxOccurs="1" minOccurs="0"/>
     </xs:sequence>
   </xs:complexType>
 

http://git-wip-us.apache.org/repos/asf/lens/blob/f0dadd79/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeFactTable.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeFactTable.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeFactTable.java
index adb6c92..896a7a1 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeFactTable.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeFactTable.java
@@ -29,10 +29,14 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.metadata.Table;
 
 import com.google.common.collect.Lists;
+import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
 
 @Slf4j
 public class CubeFactTable extends AbstractCubeTable {
+  @Getter
+  // Map<StorageName, Map<update_period, storage_table_prefix>>
+  private final Map<String, Map<UpdatePeriod, String>> storagePrefixUpdatePeriodMap;
   private String cubeName;
   private final Map<String, Set<UpdatePeriod>> storageUpdatePeriods;
 
@@ -40,8 +44,10 @@ public class CubeFactTable extends AbstractCubeTable {
     super(hiveTable);
     this.storageUpdatePeriods = getUpdatePeriods(getName(), getProperties());
     this.cubeName = getCubeName(getName(), getProperties());
+    this.storagePrefixUpdatePeriodMap = getUpdatePeriodMap(getName(), getProperties());
   }
 
+
   public CubeFactTable(String cubeName, String factName, List<FieldSchema> columns,
     Map<String, Set<UpdatePeriod>> storageUpdatePeriods) {
     this(cubeName, factName, columns, storageUpdatePeriods, 0L, new HashMap<String, String>());
@@ -54,9 +60,18 @@ public class CubeFactTable extends AbstractCubeTable {
 
   public CubeFactTable(String cubeName, String factName, List<FieldSchema> columns,
     Map<String, Set<UpdatePeriod>> storageUpdatePeriods, double weight, Map<String, String> properties) {
+    this(cubeName, factName, columns, storageUpdatePeriods, weight, properties,
+      new HashMap<String, Map<UpdatePeriod, String>>());
+
+  }
+
+  public CubeFactTable(String cubeName, String factName, List<FieldSchema> columns,
+    Map<String, Set<UpdatePeriod>> storageUpdatePeriods, double weight, Map<String, String> properties,
+    Map<String, Map<UpdatePeriod, String>> storagePrefixUpdatePeriodMap) {
     super(factName, columns, properties, weight);
     this.cubeName = cubeName;
     this.storageUpdatePeriods = storageUpdatePeriods;
+    this.storagePrefixUpdatePeriodMap = storagePrefixUpdatePeriodMap;
     addProperties();
   }
 
@@ -65,6 +80,18 @@ public class CubeFactTable extends AbstractCubeTable {
     super.addProperties();
     addCubeNames(getName(), getProperties(), cubeName);
     addUpdatePeriodProperies(getName(), getProperties(), storageUpdatePeriods);
+    addStorageTableProperties(getName(), getProperties(), storagePrefixUpdatePeriodMap);
+  }
+
+  private void addStorageTableProperties(String name, Map<String, String> properties,
+    Map<String, Map<UpdatePeriod, String>> storageUpdatePeriodMap) {
+    for (String storageName : storageUpdatePeriodMap.keySet()) {
+      String prefix = MetastoreUtil.getFactKeyPrefix(name) + "." + storageName;
+      for (Map.Entry updatePeriodEntry : storageUpdatePeriodMap.get(storageName).entrySet()) {
+        String updatePeriod = ((UpdatePeriod) updatePeriodEntry.getKey()).getName();
+        properties.put(prefix + "." + updatePeriod, (String) updatePeriodEntry.getValue());
+      }
+    }
   }
 
   private static void addUpdatePeriodProperies(String name, Map<String, String> props,
@@ -82,7 +109,29 @@ public class CubeFactTable extends AbstractCubeTable {
     props.put(MetastoreUtil.getFactCubeNameKey(factName), cubeName);
   }
 
-  private static Map<String, Set<UpdatePeriod>> getUpdatePeriods(String name, Map<String, String> props) {
+  private Map<String, Map<UpdatePeriod, String>> getUpdatePeriodMap(String factName, Map<String, String> props) {
+    Map<String, Map<UpdatePeriod, String>> ret = new HashMap<>();
+    for (Map.Entry entry : storageUpdatePeriods.entrySet()) {
+      String storage = (String) entry.getKey();
+      for (UpdatePeriod period : (Set<UpdatePeriod>) entry.getValue()) {
+        String storagePrefixKey = MetastoreUtil
+          .getUpdatePeriodStoragePrefixKey(factName.trim(), storage, period.getName());
+        String storageTableNamePrefix = props.get(storagePrefixKey);
+        if (storageTableNamePrefix == null) {
+          storageTableNamePrefix = storage;
+        }
+        Map<UpdatePeriod, String> mapOfUpdatePeriods = ret.get(storage);
+        if (mapOfUpdatePeriods == null) {
+          mapOfUpdatePeriods = new HashMap<>();
+          ret.put(storage, mapOfUpdatePeriods);
+        }
+        mapOfUpdatePeriods.put(period, storageTableNamePrefix);
+      }
+    }
+    return ret;
+  }
+
+  private Map<String, Set<UpdatePeriod>> getUpdatePeriods(String name, Map<String, String> props) {
     Map<String, Set<UpdatePeriod>> storageUpdatePeriods = new HashMap<>();
     String storagesStr = props.get(MetastoreUtil.getFactStorageListKey(name));
     if (!StringUtils.isBlank(storagesStr)) {
@@ -273,13 +322,16 @@ public class CubeFactTable extends AbstractCubeTable {
 
   /**
    * Add a storage with specified update periods
-   *
    * @param storage
    * @param updatePeriods
+   * @param updatePeriodStoragePrefix
    */
-  void addStorage(String storage, Set<UpdatePeriod> updatePeriods) {
+  void addStorage(String storage, Set<UpdatePeriod> updatePeriods,
+    Map<UpdatePeriod, String> updatePeriodStoragePrefix) {
     storageUpdatePeriods.put(storage, updatePeriods);
+    storagePrefixUpdatePeriodMap.put(storage, updatePeriodStoragePrefix);
     addUpdatePeriodProperies(getName(), getProperties(), storageUpdatePeriods);
+    addStorageTableProperties(getName(), getProperties(), storagePrefixUpdatePeriodMap);
   }
 
   /**
@@ -289,6 +341,12 @@ public class CubeFactTable extends AbstractCubeTable {
    */
   void dropStorage(String storage) {
     storageUpdatePeriods.remove(storage);
+    String prefix = MetastoreUtil.getFactKeyPrefix(getName()) + "." + storage;
+    for (Map.Entry updatePeriodEntry : storagePrefixUpdatePeriodMap.get(storage).entrySet()) {
+      String updatePeriod = ((UpdatePeriod)updatePeriodEntry.getKey()).getName();
+      getProperties().remove(prefix + "." + updatePeriod);
+    }
+    storagePrefixUpdatePeriodMap.remove(storage);
     getProperties().remove(MetastoreUtil.getFactUpdatePeriodKey(getName(), storage));
     String newStorages = StringUtils.join(storageUpdatePeriods.keySet(), ",");
     getProperties().put(MetastoreUtil.getFactStorageListKey(getName()), newStorages);
@@ -351,5 +409,7 @@ public class CubeFactTable extends AbstractCubeTable {
     return Collections.min(Lists.newArrayList(getRelativeEndTime(), getAbsoluteEndTime()));
   }
 
-
+  public String getTablePrefix(String storage, UpdatePeriod updatePeriod) {
+    return storagePrefixUpdatePeriodMap.get(storage).get(updatePeriod);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/f0dadd79/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
index 6c9cde2..087c203 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
@@ -31,7 +31,7 @@ import org.apache.lens.cube.metadata.Storage.LatestInfo;
 import org.apache.lens.cube.metadata.Storage.LatestPartColumnInfo;
 import org.apache.lens.cube.metadata.timeline.PartitionTimeline;
 import org.apache.lens.cube.metadata.timeline.PartitionTimelineFactory;
-import org.apache.lens.server.api.*;
+import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.metastore.DataCompletenessChecker;
 import org.apache.lens.server.api.util.LensUtil;
@@ -121,7 +121,13 @@ public class CubeMetastoreClient {
     if (ind <= 0) {
       throw new LensException("storageTable: " + storageTableName + ", does not belong to fact: " + fact.getName());
     }
-    return storageTableName.substring(0, ind - StorageConstants.STORGAE_SEPARATOR.length());
+    String name = storageTableName.substring(0, ind - StorageConstants.STORGAE_SEPARATOR.length());
+    for (String storageName : fact.getStorages()) {
+      if (name.equalsIgnoreCase(storageName)) {
+        return storageName;
+      }
+    }
+    throw new LensException("storageTable: " + storageTableName + ", does not belong to fact: " + fact.getName());
   }
 
   /**
@@ -169,11 +175,11 @@ public class CubeMetastoreClient {
     UpdatePeriod updatePeriod = updatePeriodStr == null ? null : UpdatePeriod.valueOf(updatePeriodStr.toUpperCase());
     List<PartitionTimeline> ret = Lists.newArrayList();
     CubeFactTable fact = getCubeFact(factName);
-    List<String> keys = Lists.newArrayList();
+    List<String> storageList = Lists.newArrayList();
     if (storage != null) {
-      keys.add(storage);
+      storageList.add(storage);
     } else {
-      keys.addAll(fact.getStorages());
+      storageList.addAll(fact.getStorages());
     }
     String partCol = null;
     if (timeDimension != null) {
@@ -186,9 +192,9 @@ public class CubeMetastoreClient {
       }
       partCol = baseCube.getPartitionColumnOfTimeDim(timeDimension);
     }
-    for (String key : keys) {
+    for (String storageName : storageList) {
       for (Map.Entry<UpdatePeriod, CaseInsensitiveStringHashMap<PartitionTimeline>> entry : partitionTimelineCache
-        .get(factName, key).entrySet()) {
+        .get(factName, storageName).entrySet()) {
         if (updatePeriod == null || entry.getKey().equals(updatePeriod)) {
           for (Map.Entry<String, PartitionTimeline> entry1 : entry.getValue().entrySet()) {
             if (partCol == null || partCol.equals(entry1.getKey())) {
@@ -201,25 +207,30 @@ public class CubeMetastoreClient {
     return ret;
   }
 
-  public void updatePartition(String fact, String storageName, Partition partition)
+  public void updatePartition(String fact, String storageName, Partition partition, UpdatePeriod updatePeriod)
     throws HiveException, InvalidOperationException, LensException {
-    updatePartitions(fact, storageName, Collections.singletonList(partition));
+    Map<UpdatePeriod, List<Partition>> updatePeriodListMap = new HashMap<>();
+    updatePeriodListMap.put(updatePeriod, Collections.singletonList(partition));
+    updatePartitions(fact, storageName, updatePeriodListMap);
   }
 
-  public void updatePartitions(String factOrDimtableName, String storageName, List<Partition> partitions)
-    throws HiveException, InvalidOperationException, LensException {
-    List<Partition> partitionsToAlter = Lists.newArrayList();
-    partitionsToAlter.addAll(partitions);
-    partitionsToAlter.addAll(getAllLatestPartsEquivalentTo(factOrDimtableName, storageName, partitions));
-    getStorage(storageName).updatePartitions(getClient(), factOrDimtableName, partitionsToAlter);
+  public void updatePartitions(String factOrDimtableName, String storageName,
+    Map<UpdatePeriod, List<Partition>> partitions) throws HiveException, InvalidOperationException, LensException {
+    for (Map.Entry entry : partitions.entrySet()) {
+      List<Partition> partitionsToAlter = Lists.newArrayList();
+      partitionsToAlter.addAll((List<Partition>) entry.getValue());
+      String storageTableName = getStorageTableName(factOrDimtableName, storageName, (UpdatePeriod) entry.getKey());
+      partitionsToAlter.addAll(
+        getAllLatestPartsEquivalentTo(factOrDimtableName, storageTableName, (List<Partition>) entry.getValue()));
+      getStorage(storageName).updatePartitions(storageTableName, getClient(), factOrDimtableName, partitionsToAlter);
+    }
   }
 
-  private List<Partition> getAllLatestPartsEquivalentTo(String factOrDimtableName, String storageName,
+  private List<Partition> getAllLatestPartsEquivalentTo(String factOrDimtableName, String storageTableName,
     List<Partition> partitions) throws HiveException, LensException {
     if (isFactTable(factOrDimtableName)) {
       return Lists.newArrayList();
     }
-    String storageTableName = getFactOrDimtableStorageTableName(factOrDimtableName, storageName);
     Table storageTable = getTable(storageTableName);
     List<String> timePartCols = getTimePartColNamesOfTable(storageTable);
     List<Partition> latestParts = Lists.newArrayList();
@@ -279,6 +290,17 @@ public class CubeMetastoreClient {
     }
   }
 
+  public void createCubeFactTable(String cubeName, String factName, List<FieldSchema> columns,
+    Map<String, Set<UpdatePeriod>> storageAggregatePeriods, double weight, Map<String, String> properties,
+    Map<String, StorageTableDesc> storageTableDescs, Map<String, Map<UpdatePeriod, String>> storageUpdatePeriodMap)
+    throws LensException {
+    CubeFactTable factTable = new CubeFactTable(cubeName, factName, columns, storageAggregatePeriods, weight,
+      properties, storageUpdatePeriodMap);
+    createCubeTable(factTable, storageTableDescs);
+    // do a get to update cache
+    getCubeFact(factName);
+
+  }
 
   /**
    * In-memory storage of {@link PartitionTimeline} objects for each valid
@@ -327,48 +349,75 @@ public class CubeMetastoreClient {
     public TreeMap<UpdatePeriod, CaseInsensitiveStringHashMap<PartitionTimeline>> get(String fact, String storage)
       throws HiveException, LensException {
       // SUSPEND CHECKSTYLE CHECK DoubleCheckedLockingCheck
-      String storageTableName = getStorageTableName(fact, Storage.getPrefix(storage));
-      if (get(storageTableName) == null) {
-        synchronized (this) {
-          if (get(storageTableName) == null) {
-            Table storageTable = getTable(storageTableName);
-            if ("true".equalsIgnoreCase(storageTable.getParameters().get(getPartitionTimelineCachePresenceKey()))) {
-              try {
-                loadTimelinesFromTableProperties(fact, storage);
-              } catch (Exception e) {
-                // Ideally this should never come. But since we have another source,
-                // let's piggyback on that for loading timeline
-                log.error("Error while loading timelines from table properties.", e);
-                loadTimelinesFromAllPartitions(fact, storage);
-              }
-            } else {
-              loadTimelinesFromAllPartitions(fact, storage);
+      // Unique key for the timeline cache, based on storageName and fact.
+      String timeLineKey = (Storage.getPrefix(storage)+ fact).toLowerCase();
+      synchronized (this) {
+        if (get(timeLineKey) == null) {
+          loadTimeLines(fact, storage, timeLineKey);
+        }
+        log.info("timeline for {} is: {}", storage, get(timeLineKey));
+        // return the final value from memory
+        return get(timeLineKey);
+        // RESUME CHECKSTYLE CHECK DoubleCheckedLockingCheck
+      }
+    }
+
+    /**
+     * @param fact
+     * @param storage
+     */
+    private void loadTimeLines(String fact, String storage, String timeLineKey) throws LensException, HiveException {
+      Set<String> uniqueStorageTables = new HashSet<>();
+      Map<UpdatePeriod, String> updatePeriodTableName = new HashMap<>();
+      for (UpdatePeriod updatePeriod : getCubeFact(fact).getUpdatePeriods().get(storage)) {
+        String storageTableName = getStorageTableName(fact, storage, updatePeriod);
+        updatePeriodTableName.put(updatePeriod, storageTableName);
+        Table storageTable = getTable(storageTableName);
+        if ("true".equalsIgnoreCase(storageTable.getParameters().get(getPartitionTimelineCachePresenceKey()))) {
+          try {
+            loadTimelinesFromTableProperties(updatePeriod, storageTableName, timeLineKey);
+          } catch (Exception e) {
+            // Ideally this should never come. But since we have another source,
+            // let's piggyback on that for loading timeline
+            log.error("Error while loading timelines from table properties.", e);
+            ensureEntryForTimeLineKey(fact, storage, updatePeriod, storageTableName, timeLineKey);
+            if (!uniqueStorageTables.contains(storageTableName)) {
+              uniqueStorageTables.add(storageTableName);
+              loadTimelinesFromAllPartitions(storageTableName, timeLineKey);
             }
           }
+        } else {
+          ensureEntryForTimeLineKey(fact, storage, updatePeriod, storageTableName, timeLineKey);
+          if (!uniqueStorageTables.contains(storageTableName)) {
+            uniqueStorageTables.add(storageTableName);
+            loadTimelinesFromAllPartitions(storageTableName, timeLineKey);
+          }
         }
-        log.info("timeline for {} is: {}", storageTableName, get(storageTableName));
       }
-      // return the final value from memory
-      return get(storageTableName);
-      // RESUME CHECKSTYLE CHECK DoubleCheckedLockingCheck
+      for (Map.Entry entry : updatePeriodTableName.entrySet()) {
+        alterTablePartitionCache(timeLineKey, (UpdatePeriod) entry.getKey(), (String) entry.getValue());
+      }
     }
 
-    private void loadTimelinesFromAllPartitions(String fact, String storage) throws HiveException, LensException {
+    private void ensureEntryForTimeLineKey(String fact, String storage, UpdatePeriod updatePeriod,
+      String storageTableName, String timeLineKey) throws LensException {
       // Not found in table properties either, compute from all partitions of the fact-storage table.
       // First make sure all combinations of update period and partition column have an entry even
       // if no partitions exist
-      String storageTableName = getStorageTableName(fact, Storage.getPrefix(storage));
-      log.info("loading from all partitions: {}", storageTableName);
-      Table storageTable = getTable(storageTableName);
-      if (getCubeFact(fact).getUpdatePeriods() != null && getCubeFact(fact).getUpdatePeriods().get(
-        storage) != null) {
-        for (UpdatePeriod updatePeriod : getCubeFact(fact).getUpdatePeriods().get(storage)) {
-          for (String partCol : getTimePartColNamesOfTable(storageTable)) {
-            ensureEntry(storageTableName, updatePeriod, partCol);
-          }
+      if (getCubeFact(fact).getUpdatePeriods() != null && getCubeFact(fact).getUpdatePeriods().get(storage) != null) {
+        log.info("loading from all partitions: {}", storageTableName);
+        Table storageTable = getTable(storageTableName);
+        for (String partCol : getTimePartColNamesOfTable(storageTable)) {
+          ensureEntry(timeLineKey, storageTableName, updatePeriod, partCol);
         }
       }
+
+    }
+
+    private void loadTimelinesFromAllPartitions(String storageTableName, String timeLineKey)
+      throws HiveException, LensException {
       // Then add all existing partitions for batch addition in respective timelines.
+      Table storageTable = getTable(storageTableName);
       List<String> timeParts = getTimePartColNamesOfTable(storageTable);
       List<FieldSchema> partCols = storageTable.getPartCols();
       for (Partition partition : getPartitionsByFilter(storageTableName, null)) {
@@ -382,23 +431,17 @@ public class CubeMetastoreClient {
         }
         for (int i = 0; i < partCols.size(); i++) {
           if (timeParts.contains(partCols.get(i).getName())) {
-            addForBatchAddition(storageTableName, period, partCols.get(i).getName(), values.get(i));
+            addForBatchAddition(timeLineKey, storageTableName, period, partCols.get(i).getName(), values.get(i));
           }
         }
       }
-      // commit all batch addition for the storage table,
-      // which will in-turn commit all batch additions in all it's timelines.
-      commitAllBatchAdditions(storageTableName);
     }
 
-    private void loadTimelinesFromTableProperties(String fact, String storage) throws HiveException, LensException {
-      // found in table properties, load from there.
-      String storageTableName = getStorageTableName(fact, Storage.getPrefix(storage));
+    private void loadTimelinesFromTableProperties(UpdatePeriod updatePeriod,
+      String storageTableName, String timeLineKey) throws HiveException, LensException {
       log.info("loading from table properties: {}", storageTableName);
-      for (UpdatePeriod updatePeriod : getCubeFact(fact).getUpdatePeriods().get(storage)) {
-        for (String partCol : getTimePartColNamesOfTable(storageTableName)) {
-          ensureEntry(storageTableName, updatePeriod, partCol).init(getTable(storageTableName));
-        }
+      for (String partCol : getTimePartColNamesOfTable(storageTableName)) {
+        ensureEntry(timeLineKey, storageTableName, updatePeriod, partCol).init(getTable(storageTableName));
       }
     }
 
@@ -406,16 +449,17 @@ public class CubeMetastoreClient {
      * Adds given partition(for storageTable, updatePeriod, partitionColum=partition) for batch addition in an
      * appropriate timeline object. Ignore if partition is not valid.
      *
-     * @param storageTable      storage table
+     * @param timeLineKey       key for the timeLine map
+     * @param storageTableName  hive table name
      * @param updatePeriod      update period
      * @param partitionColumn   partition column
      * @param partition         partition
      */
-    public void addForBatchAddition(String storageTable, UpdatePeriod updatePeriod, String partitionColumn,
-      String partition) {
+    public void addForBatchAddition(String timeLineKey, String storageTableName, UpdatePeriod updatePeriod,
+      String partitionColumn, String partition) {
       try {
-        ensureEntry(storageTable, updatePeriod, partitionColumn).addForBatchAddition(TimePartition.of(updatePeriod,
-          partition));
+        ensureEntry(timeLineKey, storageTableName, updatePeriod, partitionColumn)
+          .addForBatchAddition(TimePartition.of(updatePeriod, partition));
       } catch (LensException e) {
         // to take care of the case where partition name is something like `latest`
         log.error("Couldn't parse partition: {} with update period: {}, skipping.", partition, updatePeriod, e);
@@ -427,42 +471,24 @@ public class CubeMetastoreClient {
      * <p></p>
      * kind of like mkdir -p
      *
-     * @param storageTable    storage table
+     * @param timeLineKey    storage table
      * @param updatePeriod    update period
      * @param partitionColumn partition column
      * @return timeline if already exists, or puts a new timeline and returns.
      */
-    public PartitionTimeline ensureEntry(String storageTable, UpdatePeriod updatePeriod, String partitionColumn) {
-      if (get(storageTable) == null) {
-        put(storageTable, new TreeMap<UpdatePeriod, CaseInsensitiveStringHashMap<PartitionTimeline>>());
+    public PartitionTimeline ensureEntry(String timeLineKey, String storagTableName, UpdatePeriod updatePeriod,
+      String partitionColumn) {
+      if (get(timeLineKey) == null) {
+        put(timeLineKey, new TreeMap<UpdatePeriod, CaseInsensitiveStringHashMap<PartitionTimeline>>());
       }
-      if (get(storageTable).get(updatePeriod) == null) {
-        get(storageTable).put(updatePeriod, new CaseInsensitiveStringHashMap<PartitionTimeline>());
+      if (get(timeLineKey).get(updatePeriod) == null) {
+        get(timeLineKey).put(updatePeriod, new CaseInsensitiveStringHashMap<PartitionTimeline>());
       }
-      if (get(storageTable).get(updatePeriod).get(partitionColumn) == null) {
-        get(storageTable).get(updatePeriod).put(partitionColumn, PartitionTimelineFactory.get(
-          CubeMetastoreClient.this, storageTable, updatePeriod, partitionColumn));
-      }
-      return get(storageTable).get(updatePeriod).get(partitionColumn);
-    }
-
-    /**
-     * commit all batch addition for all its timelines.
-     *
-     * @param storageTable   storage table
-     * @throws HiveException
-     * @throws LensException
-     */
-    public void commitAllBatchAdditions(String storageTable) throws HiveException, LensException {
-      if (get(storageTable) != null) {
-        for (UpdatePeriod updatePeriod : get(storageTable).keySet()) {
-          for (String partCol : get(storageTable).get(updatePeriod).keySet()) {
-            PartitionTimeline timeline = get(storageTable).get(updatePeriod).get(partCol);
-            timeline.commitBatchAdditions();
-          }
-        }
-        alterTablePartitionCache(storageTable);
+      if (get(timeLineKey).get(updatePeriod).get(partitionColumn) == null) {
+        get(timeLineKey).get(updatePeriod).put(partitionColumn, PartitionTimelineFactory.get(
+          CubeMetastoreClient.this, storagTableName, updatePeriod, partitionColumn));
       }
+      return get(timeLineKey).get(updatePeriod).get(partitionColumn);
     }
 
     /** check partition existence in the appropriate timeline if it exists */
@@ -478,9 +504,11 @@ public class CubeMetastoreClient {
      */
     public PartitionTimeline get(String fact, String storage, UpdatePeriod updatePeriod, String partCol)
       throws HiveException, LensException {
-      return get(fact, storage) != null && get(fact, storage).get(updatePeriod) != null && get(fact, storage).get(
-        updatePeriod).get(partCol) != null ? get(fact, storage).get(updatePeriod).get(partCol) : null;
+      return get(fact, storage) != null && get(fact, storage).get(updatePeriod) != null
+               && get(fact, storage).get(updatePeriod).get(partCol) != null ? get(fact, storage).get(updatePeriod)
+               .get(partCol) : null;
     }
+
     /**
      * returns the timeline corresponding to fact-storage table, updatePeriod, partCol. throws exception if not
      * exists, which would most probably mean the combination is incorrect.
@@ -489,8 +517,8 @@ public class CubeMetastoreClient {
       throws HiveException, LensException {
       PartitionTimeline timeline = get(fact, storage, updatePeriod, partCol);
       if (timeline == null) {
-        throw new LensException(LensCubeErrorCode.TIMELINE_ABSENT.getLensErrorInfo(),
-          fact, storage, updatePeriod, partCol);
+        throw new LensException(LensCubeErrorCode.TIMELINE_ABSENT.getLensErrorInfo(), fact, storage, updatePeriod,
+          partCol);
       }
       return timeline;
     }
@@ -519,8 +547,8 @@ public class CubeMetastoreClient {
       boolean updated = false;
       for (Map.Entry<String, Date> entry : timePartSpec.entrySet()) {
         TimePartition part = TimePartition.of(updatePeriod, entry.getValue());
-        if (!partitionExistsByFilter(cubeTableName, storageName, StorageConstants.getPartFilter(entry.getKey(),
-          part.getDateString()))) {
+        if (!partitionExistsByFilter(cubeTableName, storageName, updatePeriod,
+          StorageConstants.getPartFilter(entry.getKey(), part.getDateString()))) {
           get(cubeTableName, storageName, updatePeriod, entry.getKey()).drop(part);
           updated = true;
         }
@@ -565,10 +593,10 @@ public class CubeMetastoreClient {
     Hive.closeCurrent();
   }
 
-  private void createOrAlterStorageHiveTable(Table parent, String storage, StorageTableDesc crtTblDesc)
+  private void createOrAlterStorageHiveTable(Table parent, String storageTableNamePrefix, StorageTableDesc crtTblDesc)
     throws LensException {
     try {
-      Table tbl = getStorage(storage).getStorageTable(getClient(), parent, crtTblDesc);
+      Table tbl = Storage.getStorageTable(storageTableNamePrefix, getClient(), parent, crtTblDesc);
       if (tableExists(tbl.getTableName())) {
         // alter table
         alterHiveTable(tbl.getTableName(), tbl);
@@ -730,7 +758,7 @@ public class CubeMetastoreClient {
    * @param storageAggregatePeriods Aggregate periods for the storages
    * @param weight                  Weight of the cube
    * @param properties              Properties of fact table
-   * @param storageTableDescs       Map of storage to its storage table description
+   * @param storageTableDescs       Map of storage table prefix to its storage table description
    * @throws LensException
    */
   public void createCubeFactTable(String cubeName, String factName, List<FieldSchema> columns,
@@ -808,7 +836,7 @@ public class CubeMetastoreClient {
    * Create cube table defined and create all the corresponding storage tables
    *
    * @param cubeTable         Can be fact or dimension table
-   * @param storageTableDescs Map of storage to its storage table description
+   * @param storageTableDescs Map of storage tableName prefix to its storage table description
    * @throws LensException
    */
   public void createCubeTable(AbstractCubeTable cubeTable, Map<String, StorageTableDesc> storageTableDescs)
@@ -836,14 +864,17 @@ public class CubeMetastoreClient {
    * @param fact             The CubeFactTable
    * @param storage          The storage
    * @param updatePeriods    Update periods of the fact on the storage
-   * @param storageTableDesc The storage table description
+   * @param storageTableDescs The storage table description
    * @throws LensException
    */
   public void addStorage(CubeFactTable fact, String storage, Set<UpdatePeriod> updatePeriods,
-    StorageTableDesc storageTableDesc) throws LensException {
-    fact.addStorage(storage, updatePeriods);
-    createOrAlterStorageHiveTable(getTableWithTypeFailFast(fact.getName(), CubeTableType.FACT),
-      storage, storageTableDesc);
+    Map<String, StorageTableDesc> storageTableDescs, Map<UpdatePeriod, String> updatePeriodStoragePrefix)
+    throws LensException {
+    fact.addStorage(storage, updatePeriods, updatePeriodStoragePrefix);
+    for (Map.Entry entry : storageTableDescs.entrySet()) {
+      createOrAlterStorageHiveTable(getTableWithTypeFailFast(fact.getName(), CubeTableType.FACT),
+        (String) entry.getKey(), (StorageTableDesc) entry.getValue());
+    }
     alterCubeTable(fact.getName(), getTableWithTypeFailFast(fact.getName(), CubeTableType.FACT), fact);
     updateFactCache(fact.getName());
   }
@@ -860,8 +891,8 @@ public class CubeMetastoreClient {
   public void addStorage(CubeDimensionTable dim, String storage, UpdatePeriod dumpPeriod,
     StorageTableDesc storageTableDesc) throws LensException {
     dim.alterSnapshotDumpPeriod(storage, dumpPeriod);
-    createOrAlterStorageHiveTable(getTableWithTypeFailFast(dim.getName(), CubeTableType.DIM_TABLE),
-      storage, storageTableDesc);
+    createOrAlterStorageHiveTable(getTableWithTypeFailFast(dim.getName(), CubeTableType.DIM_TABLE), storage,
+      storageTableDesc);
     alterCubeTable(dim.getName(), getTableWithTypeFailFast(dim.getName(), CubeTableType.DIM_TABLE), dim);
     updateDimCache(dim.getName());
   }
@@ -896,10 +927,19 @@ public class CubeMetastoreClient {
     return partsAdded;
   }
 
+  /**
+   * @param factOrDimTable
+   * @param storageName
+   * @param updatePeriod
+   * @param storagePartitionDescs
+   * @param type
+   * @return
+   * @throws HiveException
+   * @throws LensException
+   */
   private List<Partition> addPartitions(String factOrDimTable, String storageName, UpdatePeriod updatePeriod,
     List<StoragePartitionDesc> storagePartitionDescs, CubeTableType type) throws HiveException, LensException {
-    String storageTableName = getStorageTableName(factOrDimTable.trim(),
-      Storage.getPrefix(storageName.trim())).toLowerCase();
+    String storageTableName = getStorageTableName(factOrDimTable, storageName, updatePeriod);
     if (type == CubeTableType.DIM_TABLE) {
       // Adding partition in dimension table.
       Map<Map<String, String>, LatestInfo> latestInfos = Maps.newHashMap();
@@ -910,7 +950,7 @@ public class CubeMetastoreClient {
       }
       List<Partition> partsAdded =
         getStorage(storageName).addPartitions(getClient(), factOrDimTable, updatePeriod, storagePartitionDescs,
-          latestInfos);
+          latestInfos, storageTableName);
       ListIterator<Partition> iter = partsAdded.listIterator();
       while (iter.hasNext()) {
         if (iter.next().getSpec().values().contains(StorageConstants.LATEST_PARTITION_VALUE)) {
@@ -928,10 +968,11 @@ public class CubeMetastoreClient {
       // Adding partition in fact table.
       if (storagePartitionDescs.size() > 0) {
         partsAdded = getStorage(storageName).addPartitions(getClient(), factOrDimTable, updatePeriod,
-                storagePartitionDescs, null);
+                storagePartitionDescs, null, storageTableName);
       }
       // update hive table
-      alterTablePartitionCache(getStorageTableName(factOrDimTable, Storage.getPrefix(storageName)));
+      alterTablePartitionCache((Storage.getPrefix(storageName) + factOrDimTable).toLowerCase(), updatePeriod,
+        storageTableName);
       return partsAdded;
     } else {
       throw new LensException("Can't add partitions to anything other than fact or dimtable");
@@ -1018,20 +1059,20 @@ public class CubeMetastoreClient {
   }
 
   /**
-   * store back all timelines of given storage table to table properties
+   * store back all timelines of given storage to table properties
    *
-   * @param storageTableName  storage table name
+   * @param timeLineKey      key for the time line
+   * @param storageTableName Storage table name
    * @throws HiveException
    */
-  private void alterTablePartitionCache(String storageTableName) throws HiveException, LensException {
+  private void alterTablePartitionCache(String timeLineKey, UpdatePeriod updatePeriod, String storageTableName)
+    throws HiveException, LensException {
     Table table = getTable(storageTableName);
     Map<String, String> params = table.getParameters();
-    if (partitionTimelineCache.get(storageTableName) != null) {
-      for (UpdatePeriod updatePeriod : partitionTimelineCache.get(storageTableName).keySet()) {
-        for (Map.Entry<String, PartitionTimeline> entry : partitionTimelineCache.get(storageTableName)
-          .get(updatePeriod).entrySet()) {
-          entry.getValue().updateTableParams(table);
-        }
+    if (partitionTimelineCache.get(timeLineKey) != null) {
+      for (Map.Entry<String, PartitionTimeline> entry : partitionTimelineCache.get(timeLineKey).get(updatePeriod)
+        .entrySet()) {
+        entry.getValue().updateTableParams(table);
       }
       params.put(getPartitionTimelineCachePresenceKey(), "true");
       alterHiveTable(storageTableName, table);
@@ -1173,8 +1214,7 @@ public class CubeMetastoreClient {
    */
   public void dropPartition(String cubeTableName, String storageName, Map<String, Date> timePartSpec,
     Map<String, String> nonTimePartSpec, UpdatePeriod updatePeriod) throws HiveException, LensException {
-    String storageTableName = getStorageTableName(cubeTableName.trim(),
-      Storage.getPrefix(storageName.trim())).toLowerCase();
+    String storageTableName = getStorageTableName(cubeTableName.trim(), storageName, updatePeriod);
     Table hiveTable = getHiveTable(storageTableName);
     List<FieldSchema> partCols = hiveTable.getPartCols();
     List<String> partColNames = new ArrayList<>(partCols.size());
@@ -1244,7 +1284,8 @@ public class CubeMetastoreClient {
       // dropping fact partition
       getStorage(storageName).dropPartition(getClient(), storageTableName, partVals, null, null);
       if (partitionTimelineCache.updateForDeletion(cubeTableName, storageName, updatePeriod, timePartSpec)) {
-        this.alterTablePartitionCache(storageTableName);
+        this.alterTablePartitionCache((Storage.getPrefix(storageName) + cubeTableName).toLowerCase(), updatePeriod,
+          storageTableName);
       }
     }
   }
@@ -1277,7 +1318,7 @@ public class CubeMetastoreClient {
   public boolean factPartitionExists(String factName, String storageName, UpdatePeriod updatePeriod,
                                      Map<String, Date> partitionTimestamp,
                                      Map<String, String> partSpec) throws HiveException, LensException {
-    String storageTableName = getFactOrDimtableStorageTableName(factName, storageName);
+    String storageTableName = getStorageTableName(factName, storageName, updatePeriod);
     return partitionExists(storageTableName, updatePeriod, partitionTimestamp, partSpec);
   }
 
@@ -1286,9 +1327,9 @@ public class CubeMetastoreClient {
     return partitionExists(storageTableName, getPartitionSpec(updatePeriod, partitionTimestamps));
   }
 
-  public boolean partitionExistsByFilter(String cubeTableName, String storageName, String filter) throws LensException {
-    return partitionExistsByFilter(getStorageTableName(cubeTableName, Storage.getPrefix(storageName)),
-      filter);
+  public boolean partitionExistsByFilter(String cubeTableName, String storageName, UpdatePeriod updatePeriod,
+    String filter) throws LensException {
+    return partitionExistsByFilter(getStorageTableName(cubeTableName, storageName, updatePeriod), filter);
   }
 
   public boolean partitionExistsByFilter(String storageTableName, String filter) throws LensException {
@@ -1354,7 +1395,7 @@ public class CubeMetastoreClient {
 
   boolean latestPartitionExists(String factOrDimTblName, String storageName, String latestPartCol)
     throws HiveException, LensException {
-    String storageTableName = getStorageTableName(factOrDimTblName, Storage.getPrefix(storageName));
+    String storageTableName = MetastoreUtil.getStorageTableName(factOrDimTblName, Storage.getPrefix(storageName));
     if (isDimensionTable(factOrDimTblName)) {
       return dimTableLatestPartitionExists(storageTableName);
     } else {
@@ -2225,18 +2266,30 @@ public class CubeMetastoreClient {
    */
   public void dropStorageFromFact(String factName, String storage) throws LensException {
     CubeFactTable cft = getFactTable(factName);
+    dropHiveTablesForStorage(factName, storage);
     cft.dropStorage(storage);
-    dropHiveTable(getFactOrDimtableStorageTableName(factName, storage));
     alterCubeTable(factName, getTableWithTypeFailFast(factName, CubeTableType.FACT), cft);
     updateFactCache(factName);
   }
 
+  private void dropHiveTablesForStorage(String factName, String storage) throws LensException{
+    CubeFactTable cft = getFactTable(factName);
+    Set<String> droppedTables = new HashSet<>();
+    for (Map.Entry updatePeriodEntry : cft.getStoragePrefixUpdatePeriodMap().get(storage).entrySet()) {
+      UpdatePeriod updatePeriod = (UpdatePeriod) updatePeriodEntry.getKey();
+      String storageTableName = getStorageTableName(factName, storage, updatePeriod);
+      if (!droppedTables.contains(storageTableName)) {
+        dropHiveTable(storageTableName);
+      }
+      droppedTables.add(storageTableName);
+    }
+  }
   // updateFact will be false when fact is fully dropped
   private void dropStorageFromFact(String factName, String storage, boolean updateFact)
     throws LensException {
-    CubeFactTable cft = getFactTable(factName);
-    dropHiveTable(getFactOrDimtableStorageTableName(factName, storage));
+    dropHiveTablesForStorage(factName, storage);
     if (updateFact) {
+      CubeFactTable cft = getFactTable(factName);
       cft.dropStorage(storage);
       alterCubeTable(factName, getTableWithTypeFailFast(factName, CubeTableType.FACT), cft);
       updateFactCache(factName);
@@ -2432,4 +2485,22 @@ public class CubeMetastoreClient {
     Date now = new Date();
     return isStorageTableCandidateForRange(storageTableName, resolveDate(fromDate, now), resolveDate(toDate, now));
   }
+
+  private String getStorageTablePrefixFromStorage(String factOrDimTableName, String storage, UpdatePeriod updatePeriod)
+    throws LensException {
+    if (updatePeriod == null) {
+      return storage;
+    }
+    if (isFactTable(factOrDimTableName)) {
+      return getFactTable(factOrDimTableName).getTablePrefix(storage, updatePeriod);
+    } else {
+      return storage;
+    }
+  }
+
+  public String getStorageTableName(String factOrDimTableName, String storage, UpdatePeriod updatePeriod)
+    throws LensException {
+    return MetastoreUtil.getFactOrDimtableStorageTableName(factOrDimTableName,
+      getStorageTablePrefixFromStorage(factOrDimTableName, storage, updatePeriod));
+  }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/f0dadd79/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreUtil.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreUtil.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreUtil.java
index 53cf8af..57d4502 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreUtil.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/MetastoreUtil.java
@@ -590,4 +590,10 @@ public class MetastoreUtil {
     }
     return copy;
   }
+
+  public static String getUpdatePeriodStoragePrefixKey(String factTableName , String storageName, String updatePeriod) {
+    return MetastoreUtil.getFactKeyPrefix(factTableName) + "." + storageName + "." + updatePeriod;
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/f0dadd79/lens-cube/src/main/java/org/apache/lens/cube/metadata/Storage.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/Storage.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/Storage.java
index cd9f705..936add4 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/Storage.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/Storage.java
@@ -124,14 +124,18 @@ public abstract class Storage extends AbstractCubeTable implements PartitionMeta
   /**
    * Get the storage table descriptor for the given parent table.
    *
+   * @param storageTableNamePrefix  Storage table prefix based on update period
    * @param client The metastore client
    * @param parent Is either Fact or Dimension table
    * @param crtTbl Create table info
    * @return Table describing the storage table
    * @throws HiveException
    */
-  public Table getStorageTable(Hive client, Table parent, StorageTableDesc crtTbl) throws HiveException {
-    String storageTableName = MetastoreUtil.getStorageTableName(parent.getTableName(), this.getPrefix());
+  public static Table getStorageTable(String storageTableNamePrefix, Hive client, Table parent, StorageTableDesc crtTbl)
+    throws HiveException {
+    // Change it to the appropriate storage table name.
+    String storageTableName = MetastoreUtil
+      .getStorageTableName(parent.getTableName(), Storage.getPrefix(storageTableNamePrefix));
     Table tbl = client.getTable(storageTableName, false);
     if (tbl == null) {
       tbl = client.newTable(storageTableName);
@@ -235,21 +239,6 @@ public abstract class Storage extends AbstractCubeTable implements PartitionMeta
   }
 
   /**
-   * Add single partition to storage. Just calls #addPartitions.
-   * @param client
-   * @param addPartitionDesc
-   * @param latestInfo
-   * @throws HiveException
-   */
-  public List<Partition> addPartition(Hive client, StoragePartitionDesc addPartitionDesc, LatestInfo latestInfo)
-    throws HiveException {
-    Map<Map<String, String>, LatestInfo> latestInfos = Maps.newHashMap();
-    latestInfos.put(addPartitionDesc.getNonTimePartSpec(), latestInfo);
-    return addPartitions(client, addPartitionDesc.getCubeTableName(), addPartitionDesc.getUpdatePeriod(),
-      Collections.singletonList(addPartitionDesc), latestInfos);
-  }
-
-  /**
    * Add given partitions in the underlying hive table and update latest partition links
    *
    * @param client                hive client instance
@@ -262,12 +251,11 @@ public abstract class Storage extends AbstractCubeTable implements PartitionMeta
    */
   public List<Partition> addPartitions(Hive client, String factOrDimTable, UpdatePeriod updatePeriod,
     List<StoragePartitionDesc> storagePartitionDescs,
-    Map<Map<String, String>, LatestInfo> latestInfos) throws HiveException {
+    Map<Map<String, String>, LatestInfo> latestInfos, String tableName) throws HiveException {
     preAddPartitions(storagePartitionDescs);
     Map<Map<String, String>, Map<String, Integer>> latestPartIndexForPartCols = Maps.newHashMap();
     boolean success = false;
     try {
-      String tableName = MetastoreUtil.getStorageTableName(factOrDimTable, this.getPrefix());
       String dbName = SessionState.get().getCurrentDatabase();
       AddPartitionDesc addParts = new AddPartitionDesc(dbName, tableName, true);
       Table storageTbl = client.getTable(dbName, tableName);
@@ -383,11 +371,11 @@ public abstract class Storage extends AbstractCubeTable implements PartitionMeta
    * @throws InvalidOperationException
    * @throws HiveException
    */
-  public void updatePartitions(Hive client, String fact, List<Partition> partitions)
+  public void updatePartitions(String storageTable, Hive client, String fact, List<Partition> partitions)
     throws InvalidOperationException, HiveException {
     boolean success = false;
     try {
-      client.alterPartitions(MetastoreUtil.getFactOrDimtableStorageTableName(fact, getName()), partitions, null);
+      client.alterPartitions(storageTable, partitions, null);
       success = true;
     } finally {
       if (success) {

http://git-wip-us.apache.org/repos/asf/lens/blob/f0dadd79/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java b/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
index e21dc2a..950534c 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
@@ -28,6 +28,7 @@ import static org.apache.lens.server.api.util.LensUtil.getHashMap;
 import static org.testng.Assert.*;
 
 import java.text.SimpleDateFormat;
+
 import java.util.*;
 
 import org.apache.lens.cube.error.LensCubeErrorCode;
@@ -45,7 +46,10 @@ import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
-import org.apache.hadoop.hive.ql.metadata.*;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
@@ -965,6 +969,132 @@ public class TestCubeMetastoreClient {
     assertTrue(client.getAllFacts(altered).isEmpty());
   }
 
+  @Test(priority = 1)
+  public void testUpdatePeriodTableDescriptions() throws LensException, HiveException {
+    List<FieldSchema> factColumns = new ArrayList<>(cubeMeasures.size());
+    String factName = "testFactWithUpdatePeriodTableDescriptions";
+
+    for (CubeMeasure measure : cubeMeasures) {
+      factColumns.add(measure.getColumn());
+    }
+    // add one dimension of the cube
+    factColumns.add(new FieldSchema("zipcode", "int", "zip"));
+    FieldSchema itPart = new FieldSchema("it", "string", "date part");
+    FieldSchema etPart = new FieldSchema("et", "string", "date part");
+    String[] partColNames = new String[] { getDatePartitionKey(), itPart.getName(), etPart.getName() };
+
+    StorageTableDesc s1 = new StorageTableDesc(TextInputFormat.class, HiveIgnoreKeyTextOutputFormat.class,
+      Lists.newArrayList(getDatePartition(), itPart, etPart),
+      Lists.newArrayList(getDatePartitionKey(), itPart.getName(), etPart.getName()));
+    StorageTableDesc s2 = new StorageTableDesc(TextInputFormat.class, HiveIgnoreKeyTextOutputFormat.class,
+      Lists.newArrayList(getDatePartition(), itPart, etPart),
+      Lists.newArrayList(getDatePartitionKey(), itPart.getName(), etPart.getName()));
+
+    Map<String, Set<UpdatePeriod>> updatePeriods = getHashMap(c1, hourlyAndDaily, c2, hourlyAndDaily);
+    Map<String, StorageTableDesc> storageTables = getHashMap(HOURLY + "_" + c1, s1, DAILY + "_" + c1, s2, c2, s2);
+    Map<String, Map<UpdatePeriod, String>> storageUpdatePeriodMap = getHashMap(c1,
+      getHashMap(HOURLY, HOURLY + "_" + c1, DAILY, DAILY + "_" + c1), c2, getHashMap(HOURLY, c2, DAILY, c2));
+
+    CubeFactTable cubeFact = new CubeFactTable(CUBE_NAME, factName, factColumns, updatePeriods, 0L, null,
+      storageUpdatePeriodMap);
+    client.createCubeFactTable(CUBE_NAME, factName, factColumns, updatePeriods, 0L, null, storageTables,
+      storageUpdatePeriodMap);
+
+    assertTrue(client.tableExists(factName));
+    Table cubeTbl = client.getHiveTable(factName);
+    assertTrue(client.isFactTable(cubeTbl));
+    assertTrue(client.isFactTableForCube(cubeTbl, CUBE_NAME));
+
+    // Assert for storage tables
+    for (String entry : storageTables.keySet()) {
+      String storageTableName = getFactOrDimtableStorageTableName(factName, entry);
+      assertTrue(client.tableExists(storageTableName));
+    }
+
+    String c1TableNameHourly = getFactOrDimtableStorageTableName(cubeFact.getName(), HOURLY + "_" + c1);
+    String c2TableNameHourly = getFactOrDimtableStorageTableName(cubeFact.getName(), c2);
+
+    Table c1TableHourly = client.getHiveTable(c1TableNameHourly);
+    c1TableHourly.getParameters().put(getPartitionTimelineStorageClassKey(HOURLY, getDatePartitionKey()),
+      StoreAllPartitionTimeline.class.getCanonicalName());
+    c1TableHourly.getParameters().put(getPartitionTimelineStorageClassKey(HOURLY, itPart.getName()),
+      StoreAllPartitionTimeline.class.getCanonicalName());
+    c1TableHourly.getParameters().put(getPartitionTimelineStorageClassKey(HOURLY, etPart.getName()),
+      StoreAllPartitionTimeline.class.getCanonicalName());
+    client.pushHiveTable(c1TableHourly);
+
+    Table c2TableHourly = client.getHiveTable(c2TableNameHourly);
+    c2TableHourly.getParameters().put(getPartitionTimelineStorageClassKey(HOURLY, getDatePartitionKey()),
+      EndsAndHolesPartitionTimeline.class.getCanonicalName());
+    c2TableHourly.getParameters().put(getPartitionTimelineStorageClassKey(HOURLY, itPart.getName()),
+      EndsAndHolesPartitionTimeline.class.getCanonicalName());
+    c2TableHourly.getParameters().put(getPartitionTimelineStorageClassKey(HOURLY, etPart.getName()),
+      EndsAndHolesPartitionTimeline.class.getCanonicalName());
+    client.pushHiveTable(c2TableHourly);
+
+    assertSameTimelines(factName, new String[] { c1, c2 }, HOURLY, partColNames);
+
+    StoreAllPartitionTimeline timelineDtC1 = ((StoreAllPartitionTimeline) client.partitionTimelineCache
+      .get(factName, c1, HOURLY, getDatePartitionKey()));
+    StoreAllPartitionTimeline timelineItC1 = ((StoreAllPartitionTimeline) client.partitionTimelineCache
+      .get(factName, c1, HOURLY, itPart.getName()));
+    StoreAllPartitionTimeline timelineEtC1 = ((StoreAllPartitionTimeline) client.partitionTimelineCache
+      .get(factName, c1, HOURLY, etPart.getName()));
+    EndsAndHolesPartitionTimeline timelineDt = ((EndsAndHolesPartitionTimeline) client.partitionTimelineCache
+      .get(factName, c2, HOURLY, getDatePartitionKey()));
+    EndsAndHolesPartitionTimeline timelineIt = ((EndsAndHolesPartitionTimeline) client.partitionTimelineCache
+      .get(factName, c2, HOURLY, itPart.getName()));
+    EndsAndHolesPartitionTimeline timelineEt = ((EndsAndHolesPartitionTimeline) client.partitionTimelineCache
+      .get(factName, c2, HOURLY, etPart.getName()));
+
+    StoreAllPartitionTimeline timelineC1 = ((StoreAllPartitionTimeline) client.partitionTimelineCache
+      .get(factName, c1, HOURLY, getDatePartitionKey()));
+
+    Map<String, Date> timeParts1 = getTimePartitionByOffsets(getDatePartitionKey(), 0, itPart.getName(), 0,
+      etPart.getName(), 0);
+    StoragePartitionDesc partSpec1 = new StoragePartitionDesc(cubeFact.getName(), timeParts1, null, HOURLY);
+
+    Map<String, Date> timeParts2 = getTimePartitionByOffsets(getDatePartitionKey(), 0, etPart.getName(), 1);
+    Map<String, String> nonTimeSpec = getHashMap(itPart.getName(), "default");
+    final StoragePartitionDesc partSpec2 = new StoragePartitionDesc(cubeFact.getName(), timeParts2, nonTimeSpec,
+      HOURLY);
+
+    Map<String, Date> timeParts3 = getTimePartitionByOffsets(getDatePartitionKey(), 0, etPart.getName(), 0);
+    final StoragePartitionDesc partSpec3 = new StoragePartitionDesc(cubeFact.getName(), timeParts3, nonTimeSpec,
+      HOURLY);
+
+    client.addPartitions(Arrays.asList(partSpec1, partSpec2, partSpec3), c1, CubeTableType.FACT);
+    client.addPartitions(Arrays.asList(partSpec1, partSpec2, partSpec3), c2, CubeTableType.FACT);
+    PartitionTimeline timeline1Temp = client.partitionTimelineCache.get(factName, c1, HOURLY, getDatePartitionKey());
+    PartitionTimeline timeline2Temp = client.partitionTimelineCache.get(factName, c2, HOURLY, getDatePartitionKey());
+
+    assertEquals(timeline1Temp.getClass(), StoreAllPartitionTimeline.class);
+    assertEquals(timeline2Temp.getClass(), EndsAndHolesPartitionTimeline.class);
+
+    assertEquals(client.getAllParts(c1TableNameHourly).size(), 3);
+    assertEquals(client.getAllParts(c2TableNameHourly).size(), 3);
+
+    assertSameTimelines(factName, new String[] { c1, c2 }, HOURLY, partColNames);
+
+    assertTimeline(timelineDt, timelineDtC1, HOURLY, 0, 0);
+    assertTimeline(timelineEt, timelineEtC1, HOURLY, 0, 1);
+    assertTimeline(timelineIt, timelineItC1, HOURLY, 0, 0);
+
+    assertTrue(client.latestPartitionExists(cubeFact.getName(), c1, getDatePartitionKey()));
+    assertTrue(client.latestPartitionExists(cubeFact.getName(), c1, itPart.getName()));
+    assertTrue(client.latestPartitionExists(cubeFact.getName(), c2, etPart.getName()));
+
+    assertNoPartitionNamedLatest(c1TableNameHourly, partColNames);
+    assertNoPartitionNamedLatest(c2TableNameHourly, partColNames);
+
+    client.dropFact(factName, true);
+    assertFalse(client.tableExists(factName));
+    for (String entry : storageTables.keySet()) {
+      String storageTableName = getFactOrDimtableStorageTableName(factName, entry);
+      assertFalse(client.tableExists(storageTableName));
+    }
+  }
+
   @Test(priority = 2)
   public void testAlterDerivedCube() throws Exception {
     String name = "alter_derived_cube";
@@ -1238,7 +1368,10 @@ public class TestCubeMetastoreClient {
     s1.setFieldDelim(":");
     storageTables.put(c1, s1);
     storageTables.put(c4, s1);
-    factTable.addStorage(c4, hourlyAndDaily);
+    Map<UpdatePeriod, String> updatePeriodStoragePrefix = new HashMap<>();
+    updatePeriodStoragePrefix.put(HOURLY, c4);
+    updatePeriodStoragePrefix.put(DAILY, c4);
+    factTable.addStorage(c4, hourlyAndDaily, updatePeriodStoragePrefix);
     client.alterCubeFactTable(factName, factTable, storageTables, new HashMap<String, String>());
     CubeFactTable altered2 = client.getCubeFact(factName);
     assertTrue(client.tableExists(c1TableName));
@@ -1261,7 +1394,12 @@ public class TestCubeMetastoreClient {
     assertTrue(client.tableExists(c4TableName));
 
     // add storage
-    client.addStorage(altered2, c3, hourlyAndDaily, s1);
+    updatePeriodStoragePrefix.clear();
+    updatePeriodStoragePrefix.put(HOURLY, c3);
+    updatePeriodStoragePrefix.put(DAILY, c3);
+    Map<String, StorageTableDesc> storageTableDescMap = new HashMap<>();
+    storageTableDescMap.put(c3, s1);
+    client.addStorage(altered2, c3, hourlyAndDaily, storageTableDescMap, updatePeriodStoragePrefix);
     CubeFactTable altered3 = client.getCubeFact(factName);
     assertTrue(altered3.getStorages().contains("C3"));
     assertTrue(altered3.getUpdatePeriods().get("C3").equals(hourlyAndDaily));
@@ -1517,14 +1655,16 @@ public class TestCubeMetastoreClient {
     for (Partition partition : c1Parts) {
       partition.setLocation("blah");
       partition.setBucketCount(random.nextInt());
-      client.updatePartition(factName, c1, partition);
+      client.updatePartition(factName, c1, partition, HOURLY);
     }
     assertSamePartitions(client.getAllParts(c1TableName), c1Parts);
     for (Partition partition : c2Parts) {
       partition.setLocation("blah");
       partition.setBucketCount(random.nextInt());
     }
-    client.updatePartitions(factName, c2, c2Parts);
+    Map<UpdatePeriod, List<Partition>> partitionMap = new HashMap<>();
+    partitionMap.put(HOURLY, c2Parts);
+    client.updatePartitions(factName, c2, partitionMap);
     assertSamePartitions(client.getAllParts(c2TableName), c2Parts);
 
     assertSameTimelines(factName, storages, HOURLY, partColNames);
@@ -1998,7 +2138,6 @@ public class TestCubeMetastoreClient {
       timePartCols);
     Map<String, Set<UpdatePeriod>> updatePeriods = getHashMap(c1, updates);
     Map<String, StorageTableDesc> storageTables = getHashMap(c1, s1);
-
     CubeFactTable cubeFactWithParts = new CubeFactTable(CUBE_NAME, factNameWithPart, factColumns, updatePeriods);
 
     // create cube fact

http://git-wip-us.apache.org/repos/asf/lens/blob/f0dadd79/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
index 8b10d1d..24660e1 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
@@ -238,7 +238,7 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
       JAXBUtils.dumpPeriodsFromStorageTables(xDimTable.getStorageTables());
 
     Map<String, String> properties = JAXBUtils.mapFromXProperties(xDimTable.getProperties());
-    Map<String, StorageTableDesc> storageDesc = JAXBUtils.storageTableMapFromXStorageTables(
+    Map<String, StorageTableDesc> storageDesc = JAXBUtils.tableDescPrefixMapFromXStorageTables(
       xDimTable.getStorageTables());
 
     try (SessionContext ignored = new SessionContext(sessionid)){
@@ -289,7 +289,7 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
     try (SessionContext ignored = new SessionContext(sessionid)){
       getClient(sessionid).alterCubeDimensionTable(dimensionTable.getTableName(),
         JAXBUtils.cubeDimTableFromDimTable(dimensionTable),
-        JAXBUtils.storageTableMapFromXStorageTables(dimensionTable.getStorageTables()));
+        JAXBUtils.tableDescPrefixMapFromXStorageTables(dimensionTable.getStorageTables()));
       log.info("Updated dimension table " + dimensionTable.getTableName());
     } catch (HiveException exc) {
       throw new LensException(exc);
@@ -398,15 +398,38 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
       CubeMetastoreClient msClient = getClient(sessionid);
       CubeFactTable cft = msClient.getFactTable(fact);
       XFactTable factTable = JAXBUtils.factTableFromCubeFactTable(cft);
+      Map<String, Map<UpdatePeriod, String>> storageMap = cft.getStoragePrefixUpdatePeriodMap();
       for (String storageName : cft.getStorages()) {
         Set<UpdatePeriod> updatePeriods = cft.getUpdatePeriods().get(storageName);
-        XStorageTableElement tblElement = JAXBUtils.getXStorageTableFromHiveTable(
-          msClient.getHiveTable(MetastoreUtil.getFactOrDimtableStorageTableName(fact, storageName)));
-        tblElement.setStorageName(storageName);
-        for (UpdatePeriod p : updatePeriods) {
-          tblElement.getUpdatePeriods().getUpdatePeriod().add(XUpdatePeriod.valueOf(p.name()));
+        // This map tells if there are different tables for different update period.
+        Map<UpdatePeriod, String> updatePeriodToTableMap = storageMap.get(storageName);
+        Set<String> tableNames = new HashSet<>();
+        for (UpdatePeriod updatePeriod : updatePeriods) {
+          tableNames.add(updatePeriodToTableMap.get(updatePeriod));
+        }
+        if (tableNames.size() <= 1) {
+          XStorageTableElement tblElement = JAXBUtils.getXStorageTableFromHiveTable(
+            msClient.getHiveTable(MetastoreUtil.getFactOrDimtableStorageTableName(fact, storageName)));
+          tblElement.setStorageName(storageName);
+          for (UpdatePeriod p : updatePeriods) {
+            tblElement.getUpdatePeriods().getUpdatePeriod().add(XUpdatePeriod.valueOf(p.name()));
+          }
+          factTable.getStorageTables().getStorageTable().add(tblElement);
+        } else {
+          // Multiple storage tables.
+          XStorageTableElement tblElement = new XStorageTableElement();
+          tblElement.setStorageName(storageName);
+          XUpdatePeriods xUpdatePeriods = new XUpdatePeriods();
+          tblElement.setUpdatePeriods(xUpdatePeriods);
+          for (Map.Entry entry : updatePeriodToTableMap.entrySet()) {
+            XUpdatePeriodTableDescriptor updatePeriodTableDescriptor = new XUpdatePeriodTableDescriptor();
+            updatePeriodTableDescriptor.setTableDesc(getStorageTableDescFromHiveTable(
+              msClient.getHiveTable(MetastoreUtil.getFactOrDimtableStorageTableName(fact, (String) entry.getValue()))));
+            updatePeriodTableDescriptor.setUpdatePeriod(XUpdatePeriod.valueOf(((UpdatePeriod)entry.getKey()).name()));
+            xUpdatePeriods.getUpdatePeriodTableDescriptor().add(updatePeriodTableDescriptor);
+          }
+          factTable.getStorageTables().getStorageTable().add(tblElement);
         }
-        factTable.getStorageTables().getStorageTable().add(tblElement);
       }
       return factTable;
     }
@@ -431,7 +454,8 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
         JAXBUtils.getFactUpdatePeriodsFromStorageTables(fact.getStorageTables()),
         fact.getWeight(),
         addFactColStartTimePropertyToFactProperties(fact),
-        JAXBUtils.storageTableMapFromXStorageTables(fact.getStorageTables()));
+        JAXBUtils.tableDescPrefixMapFromXStorageTables(fact.getStorageTables()),
+        JAXBUtils.storageTablePrefixMapOfStorage(fact.getStorageTables()));
       log.info("Created fact table " + fact.getName());
     }
   }
@@ -460,7 +484,7 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   public void updateFactTable(LensSessionHandle sessionid, XFactTable fact) throws LensException {
     try (SessionContext ignored = new SessionContext(sessionid)){
       getClient(sessionid).alterCubeFactTable(fact.getName(), JAXBUtils.cubeFactFromFactTable(fact),
-        JAXBUtils.storageTableMapFromXStorageTables(fact.getStorageTables()),
+        JAXBUtils.tableDescPrefixMapFromXStorageTables(fact.getStorageTables()),
           JAXBUtils.columnStartAndEndTimeFromXColumns(fact.getColumns()));
       log.info("Updated fact table " + fact.getName());
     } catch (HiveException e) {
@@ -587,11 +611,13 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
     for (XUpdatePeriod sup : storageTable.getUpdatePeriods().getUpdatePeriod()) {
       updatePeriods.add(UpdatePeriod.valueOf(sup.name()));
     }
-    try (SessionContext ignored = new SessionContext(sessionid)){
+    try (SessionContext ignored = new SessionContext(sessionid)) {
       CubeMetastoreClient msClient = getClient(sessionid);
-      msClient.addStorage(msClient.getFactTable(fact),
-        storageTable.getStorageName(), updatePeriods,
-        JAXBUtils.storageTableDescFromXStorageTableElement(storageTable));
+      XStorageTables tables = new XStorageTables();
+      tables.getStorageTable().add(storageTable);
+      msClient.addStorage(msClient.getFactTable(fact), storageTable.getStorageName(), updatePeriods,
+        JAXBUtils.tableDescPrefixMapFromXStorageTables(tables),
+        JAXBUtils.storageTablePrefixMapOfStorage(tables).get(storageTable.getStorageName()));
       log.info("Added storage " + storageTable.getStorageName() + ":" + updatePeriods + " for fact " + fact);
     }
   }
@@ -615,17 +641,34 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
     return factTable;
   }
 
+  private Set<String> getAllTablesForStorage(LensSessionHandle sessionHandle, String fact, String storageName)
+    throws LensException {
+    Set<String> storageTableNames = new HashSet<>();
+    if (getClient(sessionHandle).isFactTable(fact)) {
+      CubeFactTable cft = getClient(sessionHandle).getCubeFact(fact);
+      Map<UpdatePeriod, String> storageMap = cft.getStoragePrefixUpdatePeriodMap().get(storageName);
+      for (Map.Entry entry : storageMap.entrySet()) {
+        storageTableNames.add(MetastoreUtil.getStorageTableName(fact, Storage.getPrefix((String) entry.getValue())));
+      }
+    } else {
+      storageTableNames.add(MetastoreUtil.getFactOrDimtableStorageTableName(fact, storageName));
+    }
+    return storageTableNames;
+  }
+
   @Override
-  public XPartitionList getAllPartitionsOfFactStorage(
-    LensSessionHandle sessionid, String fact, String storageName,
+  public XPartitionList getAllPartitionsOfFactStorage(LensSessionHandle sessionid, String fact, String storageName,
     String filter) throws LensException {
-    try (SessionContext ignored = new SessionContext(sessionid)){
+    try (SessionContext ignored = new SessionContext(sessionid)) {
       checkFactStorage(sessionid, fact, storageName);
       CubeMetastoreClient client = getClient(sessionid);
-      String storageTableName = MetastoreUtil.getFactOrDimtableStorageTableName(fact,
-        storageName);
-      List<Partition> parts = client.getPartitionsByFilter(storageTableName, filter);
-      List<String> timePartCols = client.getTimePartColNamesOfTable(storageTableName);
+      Set<String> storageTableNames = getAllTablesForStorage(sessionid, fact, storageName);
+      List<Partition> parts = new ArrayList<>();
+      List<String> timePartCols = new ArrayList<>();
+      for (String storageTableName : storageTableNames) {
+        parts.addAll(client.getPartitionsByFilter(storageTableName, filter));
+        timePartCols.addAll(client.getTimePartColNamesOfTable(storageTableName));
+      }
       return xpartitionListFromPartitionList(fact, parts, timePartCols);
     } catch (HiveException exc) {
       throw new LensException(exc);
@@ -635,10 +678,10 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   @Override
   public int addPartitionToFactStorage(LensSessionHandle sessionid, String fact, String storageName,
     XPartition partition) throws LensException {
-    try (SessionContext ignored = new SessionContext(sessionid)){
+    try (SessionContext ignored = new SessionContext(sessionid)) {
       checkFactStorage(sessionid, fact, storageName);
-      return getClient(sessionid).addPartition(storagePartSpecFromXPartition(partition), storageName,
-        CubeTableType.FACT).size();
+      return getClient(sessionid)
+        .addPartition(storagePartSpecFromXPartition(partition), storageName, CubeTableType.FACT).size();
     } catch (HiveException exc) {
       throw new LensException(exc);
     }
@@ -647,10 +690,10 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   @Override
   public int addPartitionsToFactStorage(LensSessionHandle sessionid, String fact, String storageName,
     XPartitionList partitions) throws LensException {
-    try (SessionContext ignored = new SessionContext(sessionid)){
+    try (SessionContext ignored = new SessionContext(sessionid)) {
       checkFactStorage(sessionid, fact, storageName);
-      return getClient(sessionid).addPartitions(storagePartSpecListFromXPartitionList(partitions), storageName,
-        CubeTableType.FACT).size();
+      return getClient(sessionid)
+        .addPartitions(storagePartSpecListFromXPartitionList(partitions), storageName, CubeTableType.FACT).size();
     } catch (HiveException exc) {
       throw new LensException(exc);
     }
@@ -693,15 +736,17 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   }
 
   @Override
-  public void updatePartition(LensSessionHandle sessionid, String tblName, String storageName,
-    XPartition xPartition) throws LensException {
-    try (SessionContext ignored = new SessionContext(sessionid)){
+  public void updatePartition(LensSessionHandle sessionid, String tblName, String storageName, XPartition xPartition)
+    throws LensException {
+    try (SessionContext ignored = new SessionContext(sessionid)) {
       CubeMetastoreClient client = getClient(sessionid);
-      String storageTableName = MetastoreUtil.getFactOrDimtableStorageTableName(tblName, storageName);
+      String storageTableName = client
+        .getStorageTableName(tblName, storageName, UpdatePeriod.valueOf(xPartition.getUpdatePeriod().name()));
       Partition existingPartition = client.getPartitionByFilter(storageTableName,
         StorageConstants.getPartFilter(JAXBUtils.getFullPartSpecAsMap(xPartition)));
       JAXBUtils.updatePartitionFromXPartition(existingPartition, xPartition);
-      client.updatePartition(tblName, storageName, existingPartition);
+      client.updatePartition(tblName, storageName, existingPartition,
+        UpdatePeriod.valueOf(xPartition.getUpdatePeriod().value()));
     } catch (HiveException | ClassNotFoundException | InvalidOperationException | UnsupportedOperationException exc) {
       throw new LensException(exc);
     }
@@ -710,15 +755,23 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   @Override
   public void updatePartitions(LensSessionHandle sessionid, String tblName, String storageName,
     XPartitionList xPartitions) throws LensException {
-    try (SessionContext ignored = new SessionContext(sessionid)){
+    try (SessionContext ignored = new SessionContext(sessionid)) {
       CubeMetastoreClient client = getClient(sessionid);
-      String storageTableName = MetastoreUtil.getFactOrDimtableStorageTableName(tblName, storageName);
-      List<Partition> partitionsToUpdate = new ArrayList<>(xPartitions.getPartition().size());
-      for (XPartition xPartition : xPartitions.getPartition()) {
-        Partition existingPartition = client.getPartitionByFilter(storageTableName,
-          StorageConstants.getPartFilter(JAXBUtils.getFullPartSpecAsMap(xPartition)));
-        JAXBUtils.updatePartitionFromXPartition(existingPartition, xPartition);
-        partitionsToUpdate.add(existingPartition);
+      Set<String> storageTableNames = getAllTablesForStorage(sessionid, tblName, storageName);
+      Map<UpdatePeriod, List<Partition>> partitionsToUpdate = new HashMap<>();
+      for (String storageTableName : storageTableNames) {
+        for (XPartition xPartition : xPartitions.getPartition()) {
+          Partition existingPartition = client.getPartitionByFilter(storageTableName,
+            StorageConstants.getPartFilter(JAXBUtils.getFullPartSpecAsMap(xPartition)));
+          JAXBUtils.updatePartitionFromXPartition(existingPartition, xPartition);
+          UpdatePeriod updatePeriod = UpdatePeriod.valueOf(xPartition.getUpdatePeriod().value());
+          List<Partition> partitionList = partitionsToUpdate.get(updatePeriod);
+          if (partitionList == null) {
+            partitionList = new ArrayList<>();
+            partitionsToUpdate.put(updatePeriod, partitionList);
+          }
+          partitionList.add(existingPartition);
+        }
       }
       client.updatePartitions(tblName, storageName, partitionsToUpdate);
     } catch (HiveException | ClassNotFoundException | InvalidOperationException exc) {
@@ -787,29 +840,35 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
     return period;
   }
 
-  public void dropPartitionFromStorageByValues(LensSessionHandle sessionid,
-    String cubeTableName, String storageName, String values) throws LensException {
-    try (SessionContext ignored = new SessionContext(sessionid)){
-      String tableName = MetastoreUtil.getStorageTableName(cubeTableName,
-        Storage.getPrefix(storageName));
+  public void dropPartitionFromStorageByValues(LensSessionHandle sessionid, String cubeTableName, String storageName,
+    String values) throws LensException {
+    try (SessionContext ignored = new SessionContext(sessionid)) {
+      Set<String> storageTables = getAllTablesForStorage(sessionid, cubeTableName, storageName);
+      Map<String, List<Partition>> partitions = new HashMap<>();
       CubeMetastoreClient msClient = getClient(sessionid);
-      String filter = getFilter(msClient, tableName, values);
-      List<Partition> partitions = msClient.getPartitionsByFilter(
-        tableName, filter);
-      if (partitions.size() > 1) {
-        log.error("More than one partition with specified values, correspoding filter:" + filter);
-        throw new BadRequestException("More than one partition with specified values");
-      } else if (partitions.size() == 0) {
-        log.error("No partition exists with specified values, correspoding filter:" + filter);
+      int totalPartitions = 0;
+      Partition part = null;
+      for (String tableName : storageTables) {
+        String filter = getFilter(msClient, tableName, values);
+        partitions.put(filter, msClient.getPartitionsByFilter(tableName, filter));
+        if (partitions.get(filter).size() > 1) {
+          log.error("More than one partition with specified values, corresponding filter:" + filter);
+          throw new BadRequestException("More than one partition with specified values");
+        }
+        if (partitions.get(filter).size() == 1) {
+          part = partitions.get(filter).get(0);
+        }
+        totalPartitions += partitions.get(filter).size();
+      }
+      if (totalPartitions == 0) {
+        log.error("No partition exists with specified values");
         throw new NotFoundException("No partition exists with specified values");
       }
       Map<String, Date> timeSpec = new HashMap<>();
       Map<String, String> nonTimeSpec = new HashMap<>();
-      UpdatePeriod updatePeriod = populatePartSpec(partitions.get(0), timeSpec, nonTimeSpec);
-      msClient.dropPartition(cubeTableName,
-        storageName, timeSpec, nonTimeSpec, updatePeriod);
-      log.info("Dropped partition  for dimension: " + cubeTableName
-        + " storage: " + storageName + " values:" + values);
+      UpdatePeriod updatePeriod = populatePartSpec(part, timeSpec, nonTimeSpec);
+      msClient.dropPartition(cubeTableName, storageName, timeSpec, nonTimeSpec, updatePeriod);
+      log.info("Dropped partition  for dimension: " + cubeTableName + " storage: " + storageName + " values:" + values);
     } catch (HiveException exc) {
       throw new LensException(exc);
     }
@@ -818,9 +877,12 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   public void dropPartitionFromStorageByFilter(LensSessionHandle sessionid, String cubeTableName,
     String storageName, String filter) throws LensException {
     try (SessionContext ignored = new SessionContext(sessionid)){
-      String tableName = MetastoreUtil.getStorageTableName(cubeTableName, Storage.getPrefix(storageName));
+      Set<String> storageTables = getAllTablesForStorage(sessionid, cubeTableName, storageName);
+      List<Partition> partitions  = new ArrayList<>();
       CubeMetastoreClient msClient = getClient(sessionid);
-      List<Partition> partitions = msClient.getPartitionsByFilter(tableName, filter);
+      for (String tableName : storageTables) {
+        partitions.addAll(msClient.getPartitionsByFilter(tableName, filter));
+      }
       for (Partition part : partitions) {
         try {
           Map<String, Date> timeSpec = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/lens/blob/f0dadd79/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java b/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
index 51fcb43..0bc8e77 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.mapred.InputFormat;
 
 import com.google.common.base.Optional;
 import com.google.common.collect.Maps;
-
 import lombok.extern.slf4j.Slf4j;
 
 /**
@@ -588,14 +587,22 @@ public final class JAXBUtils {
     return cols;
   }
 
-  public static Map<String, Set<UpdatePeriod>> getFactUpdatePeriodsFromStorageTables(
-    XStorageTables storageTables) {
+  public static Map<String, Set<UpdatePeriod>> getFactUpdatePeriodsFromStorageTables(XStorageTables storageTables) {
     if (storageTables != null && !storageTables.getStorageTable().isEmpty()) {
       Map<String, Set<UpdatePeriod>> factUpdatePeriods = new LinkedHashMap<String, Set<UpdatePeriod>>();
 
       for (XStorageTableElement ste : storageTables.getStorageTable()) {
-        Set<UpdatePeriod> updatePeriods = new TreeSet<UpdatePeriod>();
-        for (XUpdatePeriod upd : ste.getUpdatePeriods().getUpdatePeriod()) {
+        Set<UpdatePeriod> updatePeriods = new TreeSet<>();
+        // Check if the update period array is empty.
+        List<XUpdatePeriod> updatePeriodList = ste.getUpdatePeriods().getUpdatePeriod();
+        if (updatePeriodList.isEmpty()) {
+          List<XUpdatePeriodTableDescriptor> tableDescriptorList = ste.getUpdatePeriods()
+            .getUpdatePeriodTableDescriptor();
+          for (XUpdatePeriodTableDescriptor tableDescriptor : tableDescriptorList) {
+            updatePeriodList.add(tableDescriptor.getUpdatePeriod());
+          }
+        }
+        for (XUpdatePeriod upd : updatePeriodList) {
           updatePeriods.add(UpdatePeriod.valueOf(upd.name()));
         }
         factUpdatePeriods.put(ste.getStorageName(), updatePeriods);
@@ -706,13 +713,10 @@ public final class JAXBUtils {
 
     Map<String, Set<UpdatePeriod>> storageUpdatePeriods = getFactUpdatePeriodsFromStorageTables(
       fact.getStorageTables());
-
-    return new CubeFactTable(fact.getCubeName(),
-      fact.getName(),
-      columns,
-      storageUpdatePeriods,
-      fact.getWeight(),
-      mapFromXProperties(fact.getProperties()));
+    Map<String, Map<UpdatePeriod, String>> storageTablePrefixMap = storageTablePrefixMapOfStorage(
+      fact.getStorageTables());
+    return new CubeFactTable(fact.getCubeName(), fact.getName(), columns, storageUpdatePeriods, fact.getWeight(),
+      mapFromXProperties(fact.getProperties()), storageTablePrefixMap);
   }
 
   public static Segmentation segmentationFromXSegmentation(XSegmentation seg) throws LensException {
@@ -849,11 +853,43 @@ public final class JAXBUtils {
     return tblDesc;
   }
 
-  public static Map<String, StorageTableDesc> storageTableMapFromXStorageTables(XStorageTables storageTables) {
-    Map<String, StorageTableDesc> storageTableMap = new HashMap<String, StorageTableDesc>();
+  public static Map<String, StorageTableDesc> tableDescPrefixMapFromXStorageTables(XStorageTables storageTables) {
+    Map<String, StorageTableDesc> storageTablePrefixToDescMap = new HashMap<>();
+    if (storageTables != null && !storageTables.getStorageTable().isEmpty()) {
+      for (XStorageTableElement sTbl : storageTables.getStorageTable()) {
+        if (!sTbl.getUpdatePeriods().getUpdatePeriodTableDescriptor().isEmpty()) {
+          for (XUpdatePeriodTableDescriptor updatePeriodTable : sTbl.getUpdatePeriods()
+            .getUpdatePeriodTableDescriptor()) {
+            // Get table name with update period as the prefix.
+            storageTablePrefixToDescMap.put(updatePeriodTable.getUpdatePeriod() + "_" + sTbl.getStorageName(),
+              storageTableDescFromXStorageTableDesc(updatePeriodTable.getTableDesc()));
+          }
+        } else {
+          storageTablePrefixToDescMap.put(sTbl.getStorageName(), storageTableDescFromXStorageTableElement(sTbl));
+        }
+      }
+    }
+    return storageTablePrefixToDescMap;
+  }
+
+  public static Map<String, Map<UpdatePeriod, String>> storageTablePrefixMapOfStorage(XStorageTables storageTables) {
+    Map<String, Map<UpdatePeriod, String>> storageTableMap = new HashMap<>();
     if (storageTables != null && !storageTables.getStorageTable().isEmpty()) {
       for (XStorageTableElement sTbl : storageTables.getStorageTable()) {
-        storageTableMap.put(sTbl.getStorageName(), storageTableDescFromXStorageTableElement(sTbl));
+        Map<UpdatePeriod, String> storageNameMap = new HashMap<>();
+        if (!sTbl.getUpdatePeriods().getUpdatePeriodTableDescriptor().isEmpty()) {
+          for (XUpdatePeriodTableDescriptor updatePeriodTable : sTbl.getUpdatePeriods()
+            .getUpdatePeriodTableDescriptor()) {
+            // Get table name with update period as the prefix.
+            storageNameMap.put(UpdatePeriod.valueOf(updatePeriodTable.getUpdatePeriod().value()),
+              updatePeriodTable.getUpdatePeriod() + "_" + sTbl.getStorageName());
+          }
+        } else {
+          for (XUpdatePeriod updatePeriod :sTbl.getUpdatePeriods().getUpdatePeriod()) {
+            storageNameMap.put(UpdatePeriod.valueOf(updatePeriod.value()), sTbl.getStorageName());
+          }
+        }
+        storageTableMap.put(sTbl.getStorageName(), storageNameMap);
       }
     }
     return storageTableMap;