You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by pr...@apache.org on 2016/10/17 05:45:16 UTC

[3/4] lens git commit: LENS-743: Query retry framework for retrying upon transient failures

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryContext.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryContext.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryContext.java
index d0662f4..63d3539 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryContext.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryContext.java
@@ -27,16 +27,17 @@ import java.util.*;
 import java.util.concurrent.Future;
 
 import org.apache.lens.api.LensConf;
+import org.apache.lens.api.query.FailedAttempt;
 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.query.QueryStatus.Status;
 import org.apache.lens.server.api.LensConfConstants;
-import org.apache.lens.server.api.common.BackOffRetryHandler;
-import org.apache.lens.server.api.common.FailureContext;
 import org.apache.lens.server.api.driver.*;
 import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.query.constraint.QueryLaunchingConstraint;
+import org.apache.lens.server.api.retry.BackOffRetryHandler;
+import org.apache.lens.server.api.retry.FailureContext;
 import org.apache.lens.server.api.util.LensUtil;
 
 import org.apache.hadoop.conf.Configuration;
@@ -53,7 +54,7 @@ import lombok.extern.slf4j.Slf4j;
  * The Class QueryContext.
  */
 @Slf4j
-public class QueryContext extends AbstractQueryContext {
+public class QueryContext extends AbstractQueryContext implements FailureContext {
 
   /**
    * The Constant serialVersionUID.
@@ -191,7 +192,7 @@ public class QueryContext extends AbstractQueryContext {
   @Setter
   private byte[] queryConfHash;
 
-  transient FailureContext statusUpdateFailures = new FailureContext();
+  transient StatusUpdateFailureContext statusUpdateFailures = new StatusUpdateFailureContext();
 
   @Getter
   @Setter
@@ -200,7 +201,18 @@ public class QueryContext extends AbstractQueryContext {
   @Getter
   @Setter
   private transient Future queryLauncher;
+
   private final List<QueryDriverStatusUpdateListener> driverStatusUpdateListeners = Lists.newArrayList();
+  @Getter
+  @Setter
+  List<FailedAttempt> failedAttempts = Lists.newArrayList();
+
+  @Getter
+  @Setter
+  private BackOffRetryHandler<QueryContext> driverRetryPolicy;
+  @Getter
+  @Setter
+  private BackOffRetryHandler<QueryContext> serverRetryPolicy;
 
   /**
    * Creates context from query
@@ -224,7 +236,7 @@ public class QueryContext extends AbstractQueryContext {
    */
   public QueryContext(PreparedQueryContext prepared, String user, LensConf qconf, Configuration conf) {
     this(prepared.getUserQuery(), user, qconf, mergeConf(prepared.getConf(), conf), prepared.getDriverContext()
-        .getDriverQueryContextMap().keySet(), prepared.getDriverContext().getSelectedDriver(), true);
+      .getDriverQueryContextMap().keySet(), prepared.getDriverContext().getSelectedDriver(), true);
     setDriverContext(prepared.getDriverContext());
     setSelectedDriverQuery(prepared.getSelectedDriverQuery());
     setSelectedDriverQueryCost(prepared.getSelectedDriverQueryCost());
@@ -241,7 +253,7 @@ public class QueryContext extends AbstractQueryContext {
    * @param selectedDriver SelectedDriver
    */
   QueryContext(String userQuery, String user, LensConf qconf, Configuration conf, Collection<LensDriver> drivers,
-      LensDriver selectedDriver, boolean mergeDriverConf) {
+    LensDriver selectedDriver, boolean mergeDriverConf) {
     this(userQuery, user, qconf, conf, drivers, selectedDriver, System.currentTimeMillis(), mergeDriverConf);
   }
 
@@ -257,7 +269,7 @@ public class QueryContext extends AbstractQueryContext {
    * @param submissionTime the submission time
    */
   QueryContext(String userQuery, String user, LensConf qconf, Configuration conf, Collection<LensDriver> drivers,
-      LensDriver selectedDriver, long submissionTime, boolean mergeDriverConf) {
+    LensDriver selectedDriver, long submissionTime, boolean mergeDriverConf) {
     super(userQuery, user, qconf, conf, drivers, mergeDriverConf);
     this.submissionTime = submissionTime;
     this.queryHandle = new QueryHandle(UUID.randomUUID());
@@ -265,9 +277,9 @@ public class QueryContext extends AbstractQueryContext {
     this.lensConf = qconf;
     this.conf = conf;
     this.isPersistent = conf.getBoolean(LensConfConstants.QUERY_PERSISTENT_RESULT_SET,
-        LensConfConstants.DEFAULT_PERSISTENT_RESULT_SET);
+      LensConfConstants.DEFAULT_PERSISTENT_RESULT_SET);
     this.isDriverPersistent = conf.getBoolean(LensConfConstants.QUERY_PERSISTENT_RESULT_INDRIVER,
-        LensConfConstants.DEFAULT_DRIVER_PERSISTENT_RESULT_SET);
+      LensConfConstants.DEFAULT_DRIVER_PERSISTENT_RESULT_SET);
     this.userQuery = userQuery;
     if (selectedDriver != null) {
       this.setSelectedDriver(selectedDriver);
@@ -289,7 +301,7 @@ public class QueryContext extends AbstractQueryContext {
    * @return QueryContext object
    */
   public static QueryContext createContextWithSingleDriver(String query, String user, LensConf qconf,
-      Configuration conf, LensDriver driver, String lensSessionPublicId, boolean mergeDriverConf) {
+    Configuration conf, LensDriver driver, String lensSessionPublicId, boolean mergeDriverConf) {
     QueryContext ctx = new QueryContext(query, user, qconf, conf, Lists.newArrayList(driver), driver, mergeDriverConf);
     ctx.setLensSessionIdentifier(lensSessionPublicId);
     return ctx;
@@ -297,7 +309,7 @@ public class QueryContext extends AbstractQueryContext {
 
   public void initTransientState() {
     super.initTransientState();
-    statusUpdateFailures = new FailureContext();
+    statusUpdateFailures = new StatusUpdateFailureContext();
   }
 
   /**
@@ -338,7 +350,7 @@ public class QueryContext extends AbstractQueryContext {
       getSelectedDriverQuery(),
       status,
       resultSetPath, driverOpHandle, lensConf, submissionTime, launchTime, driverStatus.getDriverStartTime(),
-      driverStatus.getDriverFinishTime(), endTime, closedTime, queryName);
+      driverStatus.getDriverFinishTime(), endTime, closedTime, queryName, getFailedAttempts());
   }
 
   public boolean isResultAvailableInDriver() {
@@ -505,8 +517,8 @@ public class QueryContext extends AbstractQueryContext {
      *  5. rowsToPreFetch should be > 0
      */
     if (isPersistent && executeTimeoutMillis > 0
-        && result instanceof InMemoryResultSet
-        && conf.getBoolean(PREFETCH_INMEMORY_RESULTSET, DEFAULT_PREFETCH_INMEMORY_RESULTSET)) {
+      && result instanceof InMemoryResultSet
+      && conf.getBoolean(PREFETCH_INMEMORY_RESULTSET, DEFAULT_PREFETCH_INMEMORY_RESULTSET)) {
       int rowsToPreFetch = conf.getInt(PREFETCH_INMEMORY_RESULTSET_ROWS, DEFAULT_PREFETCH_INMEMORY_RESULTSET_ROWS);
       if (rowsToPreFetch > 0) {
         long executeTimeOutTime = submissionTime + executeTimeoutMillis;
@@ -579,4 +591,32 @@ public class QueryContext extends AbstractQueryContext {
       this.driverStatusUpdateListeners.add(driverStatusUpdateListener);
     }
   }
+
+  @Override
+  public long getLastFailedTime() {
+    if (getFailCount() == 0) {
+      return 0;
+    }
+    return getFailedAttempts().get(getFailedAttempts().size() - 1).getDriverFinishTime();
+  }
+
+  @Override
+  public int getFailCount() {
+    return getFailedAttempts().size();
+  }
+
+  public BackOffRetryHandler<QueryContext> getRetryPolicy() {
+    return driverRetryPolicy != null ? driverRetryPolicy : serverRetryPolicy;
+  }
+
+  public void extractFailedAttempt() {
+    extractFailedAttempt(getSelectedDriver());
+  }
+
+  public void extractFailedAttempt(LensDriver selectedDriver) {
+    failedAttempts.add(new FailedAttempt(selectedDriver.getFullyQualifiedName(), getDriverStatus().getProgress(),
+      getDriverStatus().getProgressMessage(), getDriverStatus().getErrorMessage(),
+      getDriverStatus().getDriverStartTime(), getDriverStatus().getDriverFinishTime()));
+    getDriverStatus().clear();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryEnded.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryEnded.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryEnded.java
deleted file mode 100644
index e80da6d..0000000
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryEnded.java
+++ /dev/null
@@ -1,87 +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.server.api.query;
-
-import java.util.EnumSet;
-
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryStatus;
-
-import org.apache.commons.lang.StringUtils;
-
-import lombok.Getter;
-
-/**
- * Generic event denoting that query has ended. If a listener wants to just be notified when query has ended
- * irrespective of its success or failure, then that listener can subscribe for this event type
- */
-public class QueryEnded extends StatusChange {
-
-  @Getter
-  private final QueryContext queryContext;
-  /**
-   * The user.
-   */
-  @Getter
-  private final String user;
-
-  /**
-   * The cause.
-   */
-  @Getter
-  private final String cause;
-
-  /**
-   * The Constant END_STATES.
-   */
-  public static final EnumSet<QueryStatus.Status> END_STATES = EnumSet.of(QueryStatus.Status.SUCCESSFUL,
-    QueryStatus.Status.CANCELED, QueryStatus.Status.CLOSED, QueryStatus.Status.FAILED);
-
-  /**
-   * Instantiates a new query ended.
-   *
-   * @param ctx
-   * @param eventTime the event time
-   * @param prev      the prev
-   * @param current   the current
-   * @param handle    the handle
-   * @param user      the user
-   * @param cause     the cause
-   */
-  public QueryEnded(QueryContext ctx, long eventTime, QueryStatus.Status prev, QueryStatus.Status current,
-    QueryHandle handle, String user, String cause) {
-    super(eventTime, prev, current, handle);
-    this.queryContext = ctx;
-    this.user = user;
-    this.cause = cause;
-    if (!END_STATES.contains(current)) {
-      throw new IllegalStateException("Not a valid end state: " + current + " query: " + handle);
-    }
-  }
-
-  public String toString() {
-    StringBuilder buf = new StringBuilder(super.toString());
-    if (StringUtils.isNotBlank(cause)) {
-      buf.append(" cause:").append(cause);
-    }
-    return buf.toString();
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryEvent.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryEvent.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryEvent.java
deleted file mode 100644
index 81c53be..0000000
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryEvent.java
+++ /dev/null
@@ -1,93 +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.server.api.query;
-
-import java.util.UUID;
-
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.server.api.events.LensEvent;
-
-import lombok.Getter;
-
-/**
- * A generic event related to state change of a query Subclasses must declare the specific type of change they are
- * interested in.
- * <p></p>
- * Every event will have an ID, which should be used by listeners to check if the event is already received.
- *
- * @param <T> Type of changed information about the query
- */
-public abstract class QueryEvent<T> extends LensEvent {
-
-  /**
-   * The previous value.
-   */
-  @Getter
-  protected final T previousValue;
-
-  /**
-   * The current value.
-   */
-  @Getter
-  protected final T currentValue;
-
-  /**
-   * The query handle.
-   */
-  @Getter
-  protected final QueryHandle queryHandle;
-
-  /**
-   * The id.
-   */
-  protected final UUID id = UUID.randomUUID();
-
-  /**
-   * Instantiates a new query event.
-   *
-   * @param eventTime the event time
-   * @param prev      the prev
-   * @param current   the current
-   * @param handle    the handle
-   */
-  public QueryEvent(long eventTime, T prev, T current, QueryHandle handle) {
-    super(eventTime);
-    previousValue = prev;
-    currentValue = current;
-    this.queryHandle = handle;
-  }
-
-  @Override
-  public String getEventId() {
-    return id.toString();
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * @see java.lang.Object#toString()
-   */
-  @Override
-  public String toString() {
-    StringBuilder buf = new StringBuilder("QueryEvent: ").append(getClass().getSimpleName()).append(":{id: ")
-      .append(id).append(", query:").append(getQueryHandle()).append(", change:[").append(previousValue)
-      .append(" -> ").append(currentValue).append("]}");
-    return buf.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryExecuted.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryExecuted.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryExecuted.java
deleted file mode 100644
index af8c8ee..0000000
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryExecuted.java
+++ /dev/null
@@ -1,41 +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.server.api.query;
-
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryStatus;
-
-/**
- * Event fired when query is successfully completed by the driver.
- */
-public class QueryExecuted extends StatusChange {
-
-  /**
-   * Instantiates a new query executed.
-   *
-   * @param eventTime the event time
-   * @param prev      the prev
-   * @param current   the current
-   * @param handle    the handle
-   */
-  public QueryExecuted(long eventTime, QueryStatus.Status prev, QueryStatus.Status current, QueryHandle handle) {
-    super(eventTime, prev, current, handle);
-    checkCurrentState(QueryStatus.Status.EXECUTED);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryFailed.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryFailed.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryFailed.java
deleted file mode 100644
index bdffbc2..0000000
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryFailed.java
+++ /dev/null
@@ -1,50 +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.server.api.query;
-
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryStatus;
-
-/**
- * Event fired when a query fails to execute. Use getCause() to get the reason for failure.
- */
-public class QueryFailed extends QueryEnded {
-
-  /**
-   * Instantiates a new query failed.
-   *
-   * @param ctx       the query context
-   * @param eventTime the event time
-   * @param prev      the prev
-   * @param current   the current
-   * @param handle    the handle
-   * @param user      the user
-   * @param cause     the cause
-   */
-  public QueryFailed(QueryContext ctx, long eventTime, QueryStatus.Status prev, QueryStatus.Status current,
-    QueryHandle handle,
-    String user, String cause) {
-    super(ctx, eventTime, prev, current, handle, user, cause);
-    checkCurrentState(QueryStatus.Status.FAILED);
-  }
-
-  public QueryFailed(QueryContext ctx, QueryStatus.Status prevState, QueryStatus.Status currState, String cause) {
-    this(ctx, ctx.getEndTime(), prevState, currState, ctx.getQueryHandle(), ctx.getSubmittedUser(), cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryLaunched.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryLaunched.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryLaunched.java
deleted file mode 100644
index 5fcdd73..0000000
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryLaunched.java
+++ /dev/null
@@ -1,41 +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.server.api.query;
-
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryStatus;
-
-/**
- * Event fired when query is LAUNCHED.
- */
-public class QueryLaunched extends StatusChange {
-
-  /**
-   * Instantiates a new query launched.
-   *
-   * @param eventTime the event time
-   * @param prev      the prev
-   * @param current   the current
-   * @param handle    the handle
-   */
-  public QueryLaunched(long eventTime, QueryStatus.Status prev, QueryStatus.Status current, QueryHandle handle) {
-    super(eventTime, prev, current, handle);
-    checkCurrentState(QueryStatus.Status.LAUNCHED);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryQueued.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryQueued.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryQueued.java
deleted file mode 100644
index 5f347ad..0000000
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryQueued.java
+++ /dev/null
@@ -1,59 +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.server.api.query;
-
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryStatus;
-
-/**
- * Event fired when a query is QUEUED.
- */
-public class QueryQueued extends StatusChange {
-
-  /**
-   * The user.
-   */
-  private final String user;
-
-  /**
-   * Instantiates a new query queued.
-   *
-   * @param eventTime the event time
-   * @param prev      the prev
-   * @param current   the current
-   * @param handle    the handle
-   * @param user      the user
-   */
-  public QueryQueued(long eventTime, QueryStatus.Status prev, QueryStatus.Status current, QueryHandle handle,
-    String user) {
-    super(eventTime, prev, current, handle);
-    checkCurrentState(QueryStatus.Status.QUEUED);
-    this.user = user;
-  }
-
-  /**
-   * Get the submitting user
-   *
-   * @return user
-   */
-  public final String getUser() {
-    return user;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryRejected.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryRejected.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryRejected.java
deleted file mode 100644
index c4e60e3..0000000
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryRejected.java
+++ /dev/null
@@ -1,39 +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.server.api.query;
-
-import org.apache.lens.api.query.QueryHandle;
-
-/**
- * The Class QueryRejected.
- */
-public class QueryRejected extends QueryEvent<String> {
-
-  /**
-   * Instantiates a new query rejected.
-   *
-   * @param eventTime the event time
-   * @param prev      the prev
-   * @param current   the current
-   * @param handle    the handle
-   */
-  public QueryRejected(long eventTime, String prev, String current, QueryHandle handle) {
-    super(eventTime, prev, current, handle);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryRunning.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryRunning.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryRunning.java
deleted file mode 100644
index 52aa50d..0000000
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueryRunning.java
+++ /dev/null
@@ -1,41 +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.server.api.query;
-
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryStatus;
-
-/**
- * Event fired when query enters a RUNNING state.
- */
-public class QueryRunning extends StatusChange {
-
-  /**
-   * Instantiates a new query running.
-   *
-   * @param eventTime the event time
-   * @param prev      the prev
-   * @param current   the current
-   * @param handle    the handle
-   */
-  public QueryRunning(long eventTime, QueryStatus.Status prev, QueryStatus.Status current, QueryHandle handle) {
-    super(eventTime, prev, current, handle);
-    checkCurrentState(QueryStatus.Status.RUNNING);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/QuerySuccess.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QuerySuccess.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QuerySuccess.java
deleted file mode 100644
index 298fdbb..0000000
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QuerySuccess.java
+++ /dev/null
@@ -1,47 +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.server.api.query;
-
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryStatus;
-
-/**
- * Event fired when query is successfully completed.
- */
-public class QuerySuccess extends QueryEnded {
-
-  /**
-   * Instantiates a new query success.
-   *
-   * @param ctx       the query context
-   * @param eventTime the event time
-   * @param prev      the prev
-   * @param current   the current
-   * @param handle    the handle
-   */
-  public QuerySuccess(QueryContext ctx, long eventTime, QueryStatus.Status prev, QueryStatus.Status current,
-    QueryHandle handle) {
-    super(ctx, eventTime, prev, current, handle, null, null);
-    checkCurrentState(QueryStatus.Status.SUCCESSFUL);
-  }
-
-  public QuerySuccess(QueryContext ctx, QueryStatus.Status prevState, QueryStatus.Status currState) {
-    this(ctx, ctx.getEndTime(), prevState, currState, ctx.getQueryHandle());
-  }
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueuePositionChange.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueuePositionChange.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueuePositionChange.java
deleted file mode 100644
index 062e14e..0000000
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/QueuePositionChange.java
+++ /dev/null
@@ -1,39 +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.server.api.query;
-
-import org.apache.lens.api.query.QueryHandle;
-
-/**
- * Event fired when query moves up or down in the execution engine's queue.
- */
-public class QueuePositionChange extends QueryEvent<Integer> {
-
-  /**
-   * Instantiates a new queue position change.
-   *
-   * @param eventTime the event time
-   * @param prev      the prev
-   * @param current   the current
-   * @param handle    the handle
-   */
-  public QueuePositionChange(long eventTime, Integer prev, Integer current, QueryHandle handle) {
-    super(eventTime, prev, current, handle);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/StatusChange.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/StatusChange.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/StatusChange.java
deleted file mode 100644
index 949ec20..0000000
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/StatusChange.java
+++ /dev/null
@@ -1,52 +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.server.api.query;
-
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryStatus;
-
-/**
- * The Class StatusChange.
- */
-public abstract class StatusChange extends QueryEvent<QueryStatus.Status> {
-
-  /**
-   * Instantiates a new status change.
-   *
-   * @param eventTime the event time
-   * @param prev      the prev
-   * @param current   the current
-   * @param handle    the handle
-   */
-  public StatusChange(long eventTime, QueryStatus.Status prev, QueryStatus.Status current, QueryHandle handle) {
-    super(eventTime, prev, current, handle);
-  }
-
-  /**
-   * Check current state.
-   *
-   * @param status the status
-   */
-  protected void checkCurrentState(QueryStatus.Status status) {
-    if (currentValue != status) {
-      throw new IllegalStateException("Invalid query state: " + currentValue + " query:" + queryHandle.toString());
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/StatusUpdateFailureContext.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/StatusUpdateFailureContext.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/StatusUpdateFailureContext.java
new file mode 100644
index 0000000..13ecd43
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/StatusUpdateFailureContext.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.lens.server.api.query;
+
+import org.apache.lens.server.api.retry.FailureContext;
+
+import lombok.Getter;
+
+/**
+ * Failure context captures last failure time and number of failures.
+ */
+public class StatusUpdateFailureContext implements FailureContext {
+
+  @Getter
+  private long lastFailedTime = 0;
+  @Getter
+  private int failCount = 0;
+
+  public synchronized void updateFailure() {
+    lastFailedTime = System.currentTimeMillis();
+    failCount++;
+  }
+
+  public synchronized void clear() {
+    lastFailedTime = 0;
+    failCount = 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/ChainedComparator.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/ChainedComparator.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/ChainedComparator.java
new file mode 100644
index 0000000..2cff8d8
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/ChainedComparator.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.comparators;
+
+
+import java.util.Comparator;
+import java.util.List;
+
+import lombok.Data;
+
+@Data
+public class ChainedComparator<T> implements Comparator<T> {
+  private final List<Comparator<T>> comparators;
+
+  @Override
+  public int compare(T o1, T o2) {
+    for (Comparator<T> comparator : comparators) {
+      int cmp = comparator.compare(o1, o2);
+      if (cmp != 0) {
+        return cmp;
+      }
+    }
+    return 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/FIFOQueryComparator.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/FIFOQueryComparator.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/FIFOQueryComparator.java
new file mode 100644
index 0000000..3596729
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/FIFOQueryComparator.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.comparators;
+
+import org.apache.lens.server.api.query.QueryContext;
+
+public class FIFOQueryComparator implements QueryComparator {
+
+  @Override
+  public int compare(QueryContext o1, QueryContext o2) {
+    return Long.compare(o1.getSubmissionTime(), o2.getSubmissionTime());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/MoreRetriesFirstComparator.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/MoreRetriesFirstComparator.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/MoreRetriesFirstComparator.java
new file mode 100644
index 0000000..d3242bd
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/MoreRetriesFirstComparator.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.comparators;
+
+import org.apache.lens.server.api.query.QueryContext;
+
+public class MoreRetriesFirstComparator implements QueryComparator {
+
+  @Override
+  public int compare(final QueryContext o1, final QueryContext o2) {
+    // swap order for reverse sorting
+    return Integer.compare(o2.getFailedAttempts().size(), o1.getFailedAttempts().size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryComparator.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryComparator.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryComparator.java
new file mode 100644
index 0000000..a633960
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryComparator.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.lens.server.api.query.comparators;
+
+import java.util.Comparator;
+
+import org.apache.lens.server.api.query.QueryContext;
+
+public interface QueryComparator extends Comparator<QueryContext> {
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryCostComparator.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryCostComparator.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryCostComparator.java
new file mode 100644
index 0000000..193c277
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryCostComparator.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.comparators;
+
+import org.apache.lens.server.api.query.QueryContext;
+
+public class QueryCostComparator implements QueryComparator {
+
+  @Override
+  public int compare(final QueryContext o1, final QueryContext o2) {
+    return o1.getSelectedDriverQueryCost().compareTo(o2.getSelectedDriverQueryCost());
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryPriorityComparator.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryPriorityComparator.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryPriorityComparator.java
new file mode 100644
index 0000000..7ef1a9d
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/comparators/QueryPriorityComparator.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.comparators;
+
+import org.apache.lens.server.api.query.QueryContext;
+
+public class QueryPriorityComparator implements QueryComparator {
+
+  @Override
+  public int compare(final QueryContext o1, final QueryContext o2) {
+    return o1.getPriority().compareTo(o2.getPriority());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/constraint/MaxConcurrentDriverQueriesConstraint.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/constraint/MaxConcurrentDriverQueriesConstraint.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/constraint/MaxConcurrentDriverQueriesConstraint.java
index e0f1376..8314977 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/constraint/MaxConcurrentDriverQueriesConstraint.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/constraint/MaxConcurrentDriverQueriesConstraint.java
@@ -41,22 +41,50 @@ public class MaxConcurrentDriverQueriesConstraint implements QueryLaunchingConst
   private final int maxConcurrentLaunches;
 
   @Override
-  public boolean allowsLaunchOf(
+  public String allowsLaunchOf(
     final QueryContext candidateQuery, final EstimatedImmutableQueryCollection launchedQueries) {
 
     final LensDriver selectedDriver = candidateQuery.getSelectedDriver();
     final Set<QueryContext> driverLaunchedQueries = launchedQueries.getQueries(selectedDriver);
-    final boolean canLaunch = (launchedQueries.getQueriesCount(selectedDriver) < maxConcurrentQueries)
-      && (getIsLaunchingCount(driverLaunchedQueries) < maxConcurrentLaunches)
-      && canLaunchWithQueueConstraint(candidateQuery, driverLaunchedQueries)
-      && canLaunchWithPriorityConstraint(candidateQuery, driverLaunchedQueries);
-    log.debug("canLaunch:{}", canLaunch);
-    return canLaunch;
+
+    String maxConcurrentLimitation = canLaunchWithMaxConcurrentConstraint(candidateQuery,
+      launchedQueries.getQueriesCount(selectedDriver));
+    if (maxConcurrentLimitation != null) {
+      return maxConcurrentLimitation;
+    }
+    String maxLaunchingLimitation = canLaunchWithMaxLaunchingConstraint(driverLaunchedQueries);
+    if (maxLaunchingLimitation != null) {
+      return maxLaunchingLimitation;
+    }
+    String queueLimitation = canLaunchWithQueueConstraint(candidateQuery, driverLaunchedQueries);
+    if (queueLimitation != null) {
+      return queueLimitation;
+    }
+    String priorityLimitation = canLaunchWithPriorityConstraint(candidateQuery, driverLaunchedQueries);
+    if (priorityLimitation != null) {
+      return priorityLimitation;
+    }
+    return null;
+  }
+
+  private String canLaunchWithMaxLaunchingConstraint(Set<QueryContext> driverLaunchedQueries) {
+    int launchingCount = getIsLaunchingCount(driverLaunchedQueries);
+    if (launchingCount >= maxConcurrentLaunches) {
+      return launchingCount + "/" + maxConcurrentLaunches + " launches happening";
+    }
+    return null;
   }
 
-  private boolean canLaunchWithQueueConstraint(QueryContext candidateQuery, Set<QueryContext> launchedQueries) {
+  private String canLaunchWithMaxConcurrentConstraint(QueryContext candidateQuery, int concurrentLaunched) {
+    if (concurrentLaunched >= maxConcurrentQueries) {
+      return concurrentLaunched + "/" + maxConcurrentQueries + " queries running on "
+        + candidateQuery.getSelectedDriver().getFullyQualifiedName();
+    }
+    return null;
+  }
+  private String canLaunchWithQueueConstraint(QueryContext candidateQuery, Set<QueryContext> launchedQueries) {
     if (maxConcurrentQueriesPerQueue == null) {
-      return true;
+      return null;
     }
     String queue = candidateQuery.getQueue();
     Integer limit = maxConcurrentQueriesPerQueue.get(queue);
@@ -64,7 +92,7 @@ public class MaxConcurrentDriverQueriesConstraint implements QueryLaunchingConst
       if (defaultMaxConcurrentQueriesPerQueueLimit != null) { //Check if any default limit is enabled for all queues
         limit = defaultMaxConcurrentQueriesPerQueueLimit;
       } else {
-        return true;
+        return null;
       }
     }
     int launchedOnQueue = 0;
@@ -73,17 +101,20 @@ public class MaxConcurrentDriverQueriesConstraint implements QueryLaunchingConst
         launchedOnQueue++;
       }
     }
-    return launchedOnQueue < limit;
+    if (launchedOnQueue >= limit) {
+      return launchedOnQueue + "/" + limit + " queries running in Queue " + queue;
+    }
+    return null;
   }
 
-  private boolean canLaunchWithPriorityConstraint(QueryContext candidateQuery, Set<QueryContext> launchedQueries) {
+  private String canLaunchWithPriorityConstraint(QueryContext candidateQuery, Set<QueryContext> launchedQueries) {
     if (maxConcurrentQueriesPerPriority == null) {
-      return true;
+      return null;
     }
     Priority priority = candidateQuery.getPriority();
     Integer limit = maxConcurrentQueriesPerPriority.get(priority);
     if (limit == null) {
-      return true;
+      return null;
     }
     int launchedOnPriority = 0;
     for (QueryContext context : launchedQueries) {
@@ -91,7 +122,10 @@ public class MaxConcurrentDriverQueriesConstraint implements QueryLaunchingConst
         launchedOnPriority++;
       }
     }
-    return launchedOnPriority < limit;
+    if (launchedOnPriority >= limit) {
+      return launchedOnPriority + "/" + limit + " queries running with priority " + priority;
+    }
+    return null;
   }
 
   private int getIsLaunchingCount(final Set<QueryContext> launchedQueries) {

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/constraint/QueryLaunchingConstraint.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/constraint/QueryLaunchingConstraint.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/constraint/QueryLaunchingConstraint.java
index 12d9562..29142eb 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/constraint/QueryLaunchingConstraint.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/constraint/QueryLaunchingConstraint.java
@@ -28,7 +28,7 @@ public interface QueryLaunchingConstraint {
    *
    * @param candidateQuery The query which is the next candidate to be launched.
    * @param launchedQueries Current launched queries
-   * @return
+   * @return null if allowed to launch, otherwise a String containing the reason to block launch
    */
-  boolean allowsLaunchOf(final QueryContext candidateQuery, final EstimatedImmutableQueryCollection launchedQueries);
+  String allowsLaunchOf(final QueryContext candidateQuery, final EstimatedImmutableQueryCollection launchedQueries);
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/PriorityChange.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/PriorityChange.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/PriorityChange.java
new file mode 100644
index 0000000..72d5eb7
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/PriorityChange.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.Priority;
+import org.apache.lens.api.query.QueryHandle;
+
+/**
+ * Event fired when query priority changes.
+ */
+public class PriorityChange extends QueryEvent<Priority> {
+
+  /**
+   * Instantiates a new priority change.
+   *
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   */
+  public PriorityChange(long eventTime, Priority prev, Priority current, QueryHandle handle) {
+    super(eventTime, prev, current, handle);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryAccepted.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryAccepted.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryAccepted.java
new file mode 100644
index 0000000..5ab7cb4
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryAccepted.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+
+/**
+ * The Class QueryAccepted.
+ */
+public class QueryAccepted extends QueryEvent<String> {
+
+  /**
+   * Instantiates a new query accepted.
+   *
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   */
+  public QueryAccepted(long eventTime, String prev, String current, QueryHandle handle) {
+    super(eventTime, prev, current, handle);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryCancelled.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryCancelled.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryCancelled.java
new file mode 100644
index 0000000..9198e02
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryCancelled.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+import org.apache.lens.server.api.query.QueryContext;
+
+/**
+ * Event fired when query is cancelled.
+ */
+public class QueryCancelled extends QueryEnded {
+
+  /**
+   * Instantiates a new query cancelled.
+   *
+   * @param ctx       the query context
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   * @param user      the user
+   * @param cause     the cause
+   */
+  public QueryCancelled(QueryContext ctx, long eventTime, QueryStatus.Status prev, QueryStatus.Status current,
+    QueryHandle handle,
+    String user, String cause) {
+    super(ctx, eventTime, prev, current, handle, user, cause);
+    checkCurrentState(QueryStatus.Status.CANCELED);
+  }
+
+  public QueryCancelled(QueryContext ctx, QueryStatus.Status prevState, QueryStatus.Status currState, String cause) {
+    // TODO: correct username. put who cancelled it, not the submitter. Similar for others
+    this(ctx, ctx.getEndTime(), prevState, currState, ctx.getQueryHandle(), ctx.getSubmittedUser(), cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryClosed.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryClosed.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryClosed.java
new file mode 100644
index 0000000..289159e
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryClosed.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+import org.apache.lens.server.api.query.QueryContext;
+
+/**
+ * Event fired when a query is closed.
+ */
+public class QueryClosed extends QueryEnded {
+
+  /**
+   * Instantiates a new query closed.
+   *
+   * @param ctx       the query context
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   * @param user      the user
+   * @param cause     the cause
+   */
+  public QueryClosed(QueryContext ctx, long eventTime, QueryStatus.Status prev, QueryStatus.Status current,
+    QueryHandle handle,
+    String user, String cause) {
+    super(ctx, eventTime, prev, current, handle, user, cause);
+    checkCurrentState(QueryStatus.Status.CLOSED);
+  }
+
+  public QueryClosed(QueryContext ctx, QueryStatus.Status prevState, QueryStatus.Status currState, String cause) {
+    this(ctx, ctx.getClosedTime(), prevState, currState, ctx.getQueryHandle(), ctx.getSubmittedUser(), cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryEnded.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryEnded.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryEnded.java
new file mode 100644
index 0000000..cf052ef
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryEnded.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import java.util.EnumSet;
+
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+import org.apache.lens.server.api.query.QueryContext;
+
+import org.apache.commons.lang.StringUtils;
+
+import lombok.Getter;
+
+/**
+ * Generic event denoting that query has ended. If a listener wants to just be notified when query has ended
+ * irrespective of its success or failure, then that listener can subscribe for this event type
+ */
+public class QueryEnded extends StatusChange {
+
+  @Getter
+  private final QueryContext queryContext;
+  /**
+   * The user.
+   */
+  @Getter
+  private final String user;
+
+  /**
+   * The cause.
+   */
+  @Getter
+  private final String cause;
+
+  /**
+   * The Constant END_STATES.
+   */
+  public static final EnumSet<QueryStatus.Status> END_STATES = EnumSet.of(QueryStatus.Status.SUCCESSFUL,
+    QueryStatus.Status.CANCELED, QueryStatus.Status.CLOSED, QueryStatus.Status.FAILED);
+
+  /**
+   * Instantiates a new query ended.
+   *
+   * @param ctx
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   * @param user      the user
+   * @param cause     the cause
+   */
+  public QueryEnded(QueryContext ctx, long eventTime, QueryStatus.Status prev, QueryStatus.Status current,
+    QueryHandle handle, String user, String cause) {
+    super(eventTime, prev, current, handle);
+    this.queryContext = ctx;
+    this.user = user;
+    this.cause = cause;
+    if (!END_STATES.contains(current)) {
+      throw new IllegalStateException("Not a valid end state: " + current + " query: " + handle);
+    }
+  }
+
+  public String toString() {
+    StringBuilder buf = new StringBuilder(super.toString());
+    if (StringUtils.isNotBlank(cause)) {
+      buf.append(" cause:").append(cause);
+    }
+    return buf.toString();
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryEvent.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryEvent.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryEvent.java
new file mode 100644
index 0000000..f577b7d
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryEvent.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import java.util.UUID;
+
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.server.api.events.LensEvent;
+
+import lombok.Getter;
+
+/**
+ * A generic event related to state change of a query Subclasses must declare the specific type of change they are
+ * interested in.
+ * <p></p>
+ * Every event will have an ID, which should be used by listeners to check if the event is already received.
+ *
+ * @param <T> Type of changed information about the query
+ */
+public abstract class QueryEvent<T> extends LensEvent {
+
+  /**
+   * The previous value.
+   */
+  @Getter
+  protected final T previousValue;
+
+  /**
+   * The current value.
+   */
+  @Getter
+  protected final T currentValue;
+
+  /**
+   * The query handle.
+   */
+  @Getter
+  protected final QueryHandle queryHandle;
+
+  /**
+   * The id.
+   */
+  protected final UUID id = UUID.randomUUID();
+
+  /**
+   * Instantiates a new query event.
+   *
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   */
+  public QueryEvent(long eventTime, T prev, T current, QueryHandle handle) {
+    super(eventTime);
+    previousValue = prev;
+    currentValue = current;
+    this.queryHandle = handle;
+  }
+
+  @Override
+  public String getEventId() {
+    return id.toString();
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    StringBuilder buf = new StringBuilder("QueryEvent: ").append(getClass().getSimpleName()).append(":{id: ")
+      .append(id).append(", query:").append(getQueryHandle()).append(", change:[").append(previousValue)
+      .append(" -> ").append(currentValue).append("]}");
+    return buf.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryExecuted.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryExecuted.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryExecuted.java
new file mode 100644
index 0000000..b4f2b37
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryExecuted.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+
+/**
+ * Event fired when query is successfully completed by the driver.
+ */
+public class QueryExecuted extends StatusChange {
+
+  /**
+   * Instantiates a new query executed.
+   *
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   */
+  public QueryExecuted(long eventTime, QueryStatus.Status prev, QueryStatus.Status current, QueryHandle handle) {
+    super(eventTime, prev, current, handle);
+    checkCurrentState(QueryStatus.Status.EXECUTED);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryFailed.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryFailed.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryFailed.java
new file mode 100644
index 0000000..14a868a
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryFailed.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+import org.apache.lens.server.api.query.QueryContext;
+
+/**
+ * Event fired when a query fails to execute. Use getCause() to get the reason for failure.
+ */
+public class QueryFailed extends QueryEnded {
+
+  /**
+   * Instantiates a new query failed.
+   *
+   * @param ctx       the query context
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   * @param user      the user
+   * @param cause     the cause
+   */
+  public QueryFailed(QueryContext ctx, long eventTime, QueryStatus.Status prev, QueryStatus.Status current,
+    QueryHandle handle,
+    String user, String cause) {
+    super(ctx, eventTime, prev, current, handle, user, cause);
+    checkCurrentState(QueryStatus.Status.FAILED);
+  }
+
+  public QueryFailed(QueryContext ctx, QueryStatus.Status prevState, QueryStatus.Status currState, String cause) {
+    this(ctx, ctx.getEndTime(), prevState, currState, ctx.getQueryHandle(), ctx.getSubmittedUser(), cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryLaunched.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryLaunched.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryLaunched.java
new file mode 100644
index 0000000..11f76da
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryLaunched.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+
+/**
+ * Event fired when query is LAUNCHED.
+ */
+public class QueryLaunched extends StatusChange {
+
+  /**
+   * Instantiates a new query launched.
+   *
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   */
+  public QueryLaunched(long eventTime, QueryStatus.Status prev, QueryStatus.Status current, QueryHandle handle) {
+    super(eventTime, prev, current, handle);
+    checkCurrentState(QueryStatus.Status.LAUNCHED);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryQueued.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryQueued.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryQueued.java
new file mode 100644
index 0000000..57bd817
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryQueued.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+
+/**
+ * Event fired when a query is QUEUED.
+ */
+public class QueryQueued extends StatusChange {
+
+  /**
+   * The user.
+   */
+  private final String user;
+
+  /**
+   * Instantiates a new query queued.
+   *
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   * @param user      the user
+   */
+  public QueryQueued(long eventTime, QueryStatus.Status prev, QueryStatus.Status current, QueryHandle handle,
+    String user) {
+    super(eventTime, prev, current, handle);
+    checkCurrentState(QueryStatus.Status.QUEUED);
+    this.user = user;
+  }
+
+  /**
+   * Get the submitting user
+   *
+   * @return user
+   */
+  public final String getUser() {
+    return user;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryQueuedForRetry.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryQueuedForRetry.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryQueuedForRetry.java
new file mode 100644
index 0000000..e90e743
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryQueuedForRetry.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+
+/**
+ * Event fired when a query is QUEUED.
+ */
+public class QueryQueuedForRetry extends QueryQueued {
+
+  /**
+   * Instantiates a new query queued.
+   *  @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   * @param user      the user
+   */
+  public QueryQueuedForRetry(long eventTime, QueryStatus.Status prev, QueryStatus.Status current, QueryHandle handle,
+    String user) {
+    super(eventTime, prev, current, handle, user);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryRejected.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryRejected.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryRejected.java
new file mode 100644
index 0000000..1a0b867
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryRejected.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+
+/**
+ * The Class QueryRejected.
+ */
+public class QueryRejected extends QueryEvent<String> {
+
+  /**
+   * Instantiates a new query rejected.
+   *
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   */
+  public QueryRejected(long eventTime, String prev, String current, QueryHandle handle) {
+    super(eventTime, prev, current, handle);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryRunning.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryRunning.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryRunning.java
new file mode 100644
index 0000000..c865412
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueryRunning.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+
+/**
+ * Event fired when query enters a RUNNING state.
+ */
+public class QueryRunning extends StatusChange {
+
+  /**
+   * Instantiates a new query running.
+   *
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   */
+  public QueryRunning(long eventTime, QueryStatus.Status prev, QueryStatus.Status current, QueryHandle handle) {
+    super(eventTime, prev, current, handle);
+    checkCurrentState(QueryStatus.Status.RUNNING);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QuerySuccess.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QuerySuccess.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QuerySuccess.java
new file mode 100644
index 0000000..9e08ba4
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QuerySuccess.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryStatus;
+import org.apache.lens.server.api.query.QueryContext;
+
+/**
+ * Event fired when query is successfully completed.
+ */
+public class QuerySuccess extends QueryEnded {
+
+  /**
+   * Instantiates a new query success.
+   *
+   * @param ctx       the query context
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   */
+  public QuerySuccess(QueryContext ctx, long eventTime, QueryStatus.Status prev, QueryStatus.Status current,
+    QueryHandle handle) {
+    super(ctx, eventTime, prev, current, handle, null, null);
+    checkCurrentState(QueryStatus.Status.SUCCESSFUL);
+  }
+
+  public QuerySuccess(QueryContext ctx, QueryStatus.Status prevState, QueryStatus.Status currState) {
+    this(ctx, ctx.getEndTime(), prevState, currState, ctx.getQueryHandle());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/38ab6c60/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueuePositionChange.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueuePositionChange.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueuePositionChange.java
new file mode 100644
index 0000000..bbc6f4f
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/events/QueuePositionChange.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.query.events;
+
+import org.apache.lens.api.query.QueryHandle;
+
+/**
+ * Event fired when query moves up or down in the execution engine's queue.
+ */
+public class QueuePositionChange extends QueryEvent<Integer> {
+
+  /**
+   * Instantiates a new queue position change.
+   *
+   * @param eventTime the event time
+   * @param prev      the prev
+   * @param current   the current
+   * @param handle    the handle
+   */
+  public QueuePositionChange(long eventTime, Integer prev, Integer current, QueryHandle handle) {
+    super(eventTime, prev, current, handle);
+  }
+}