You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by hi...@apache.org on 2015/06/20 12:07:27 UTC

[1/2] incubator-lens git commit: LENS-487: Implementation for consistent error response display for Lens Client

Repository: incubator-lens
Updated Branches:
  refs/heads/master 5d20751d3 -> d1d997080


http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-regression/src/main/java/org/apache/lens/regression/core/helpers/QueryHelper.java
----------------------------------------------------------------------
diff --git a/lens-regression/src/main/java/org/apache/lens/regression/core/helpers/QueryHelper.java b/lens-regression/src/main/java/org/apache/lens/regression/core/helpers/QueryHelper.java
index 6b8bdda..8957aef 100644
--- a/lens-regression/src/main/java/org/apache/lens/regression/core/helpers/QueryHelper.java
+++ b/lens-regression/src/main/java/org/apache/lens/regression/core/helpers/QueryHelper.java
@@ -29,7 +29,7 @@ import javax.xml.bind.JAXBException;
 
 import org.apache.lens.api.LensConf;
 import org.apache.lens.api.query.*;
-import org.apache.lens.api.response.LensResponse;
+import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.regression.core.constants.QueryURL;
 import org.apache.lens.regression.core.type.FormBuilder;
 import org.apache.lens.regression.core.type.MapBuilder;
@@ -83,7 +83,7 @@ public class QueryHelper extends ServiceManagerHelper {
     AssertUtil.assertSucceededResponse(response);
     String queryHandleString = response.readEntity(String.class);
     logger.info(queryHandleString);
-    LensResponse successResponse = (LensResponse) Util.getObject(queryHandleString, LensResponse.class);
+    LensAPIResult successResponse = (LensAPIResult) Util.getObject(queryHandleString, LensAPIResult.class);
     QueryHandle queryHandle = (QueryHandle) successResponse.getData();
     if (queryHandle == null) {
       throw new LensException("Query Execute Failed");
@@ -138,7 +138,7 @@ public class QueryHelper extends ServiceManagerHelper {
     AssertUtil.assertSucceededResponse(response);
     String queryHandleString = response.readEntity(String.class);
     logger.info(queryHandleString);
-    LensResponse successResponse = (LensResponse) Util.getObject(queryHandleString, LensResponse.class);
+    LensAPIResult successResponse = (LensAPIResult) Util.getObject(queryHandleString, LensAPIResult.class);
     QueryHandleWithResultSet queryHandleWithResultSet = (QueryHandleWithResultSet) successResponse.getData();
     if (queryHandleWithResultSet==null) {
       throw new LensException("Query Execute Failed");
@@ -200,7 +200,7 @@ public class QueryHelper extends ServiceManagerHelper {
     AssertUtil.assertSucceededResponse(response);
     String queryHandleString = response.readEntity(String.class);
     logger.info(queryHandleString);
-    LensResponse successResponse = (LensResponse) Util.getObject(queryHandleString, LensResponse.class);
+    LensAPIResult successResponse = (LensAPIResult) Util.getObject(queryHandleString, LensAPIResult.class);
     QueryHandle queryHandle = (QueryHandle) successResponse.getData();
     return queryHandle;
   }
@@ -228,7 +228,7 @@ public class QueryHelper extends ServiceManagerHelper {
     AssertUtil.assertSucceededResponse(response);
     String queryPlanString = response.readEntity(String.class);
     logger.info(queryPlanString);
-    LensResponse successResponse = (LensResponse) Util.getObject(queryPlanString, LensResponse.class);
+    LensAPIResult successResponse = (LensAPIResult) Util.getObject(queryPlanString, LensAPIResult.class);
     QueryPlan queryPlan = (QueryPlan) successResponse.getData();
     return queryPlan;
   }
@@ -265,7 +265,7 @@ public class QueryHelper extends ServiceManagerHelper {
     AssertUtil.assertSucceededResponse(response);
     String queryCostString = response.readEntity(String.class);
     logger.info(queryCostString);
-    LensResponse successResponse = (LensResponse) Util.getObject(queryCostString, LensResponse.class);
+    LensAPIResult successResponse = (LensAPIResult) Util.getObject(queryCostString, LensAPIResult.class);
     QueryCost queryCost = (QueryCost) successResponse.getData();
     if (queryCost == null) {
       throw new LensException("Estimate Failed");

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server-api/src/main/java/org/apache/lens/server/api/driver/DriverQueryStatus.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/driver/DriverQueryStatus.java b/lens-server-api/src/main/java/org/apache/lens/server/api/driver/DriverQueryStatus.java
index 2c35276..f78b7c3 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/driver/DriverQueryStatus.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/driver/DriverQueryStatus.java
@@ -167,7 +167,7 @@ public class DriverQueryStatus implements Serializable {
       break;
     }
 
-    return new QueryStatus(progress, qstate, statusMessage, isResultSetAvailable, progressMessage, errorMessage);
+    return new QueryStatus(progress, qstate, statusMessage, isResultSetAvailable, progressMessage, errorMessage, null);
   }
 
   /**
@@ -179,7 +179,7 @@ public class DriverQueryStatus implements Serializable {
    */
   public static QueryStatus createQueryStatus(QueryStatus.Status state, DriverQueryStatus dstatus) {
     return new QueryStatus(dstatus.progress, state, dstatus.statusMessage, dstatus.isResultSetAvailable,
-      dstatus.progressMessage, dstatus.errorMessage);
+      dstatus.progressMessage, dstatus.errorMessage, null);
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java b/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
index 9d8748c..49b118d 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
@@ -29,8 +29,8 @@ import java.util.Arrays;
 
 import org.apache.lens.api.error.ErrorCollection;
 import org.apache.lens.api.error.LensError;
-import org.apache.lens.api.response.LensErrorTO;
-import org.apache.lens.api.response.LensResponse;
+import org.apache.lens.api.result.LensAPIResult;
+import org.apache.lens.api.result.LensErrorTO;
 
 import org.apache.commons.lang.exception.ExceptionUtils;
 
@@ -53,7 +53,7 @@ public class LensException extends Exception {
    * The lensResponse prepared by {@link #buildLensErrorResponse(ErrorCollection, String, String)}
    * */
   @Getter
-  private LensResponse lensResponse;
+  private LensAPIResult lensAPIResult;
 
   /**
    * Constructs a new Lens Exception.
@@ -138,7 +138,7 @@ public class LensException extends Exception {
 
     final LensError lensError = errorCollection.getLensError(errorCode);
     final LensErrorTO lensErrorTO = buildLensErrorTO(errorCollection, lensError);
-    lensResponse = LensResponse.composedOf(apiVersion, id, lensErrorTO, lensError.getHttpStatusCode());
+    lensAPIResult = LensAPIResult.composedOf(apiVersion, id, lensErrorTO, lensError.getHttpStatusCode());
   }
 
   public final LensErrorTO buildLensErrorTO(final ErrorCollection errorCollection) {

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensMultiCauseException.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensMultiCauseException.java b/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensMultiCauseException.java
index eb0df4d..af9e33a 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensMultiCauseException.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensMultiCauseException.java
@@ -26,7 +26,7 @@ import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.lens.api.error.ErrorCollection;
-import org.apache.lens.api.response.LensErrorTO;
+import org.apache.lens.api.result.LensErrorTO;
 
 import com.google.common.collect.ImmutableList;
 import lombok.AccessLevel;

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server-api/src/main/java/org/apache/lens/server/api/query/FinishedLensQuery.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/FinishedLensQuery.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/FinishedLensQuery.java
index 9ba1843..6e7ab5d 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/FinishedLensQuery.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/FinishedLensQuery.java
@@ -191,7 +191,7 @@ public class FinishedLensQuery {
     qctx.setQueryHandle(QueryHandle.fromString(handle));
     qctx.setEndTime(getEndTime());
     qctx.setStatusSkippingTransitionTest(new QueryStatus(0.0, QueryStatus.Status.valueOf(getStatus()),
-        getErrorMessage() == null ? "" : getErrorMessage(), getResult() != null, null, null));
+        getErrorMessage() == null ? "" : getErrorMessage(), getResult() != null, null, null, null));
     qctx.getDriverStatus().setDriverStartTime(getDriverStartTime());
     qctx.getDriverStatus().setDriverFinishTime(getDriverEndTime());
     qctx.setResultSetPath(getResult());

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/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 3e0c26c..2d3a19a 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
@@ -213,7 +213,7 @@ public class QueryContext extends AbstractQueryContext implements Comparable<Que
     super(userQuery, user, qconf, conf, drivers, mergeDriverConf);
     this.submissionTime = submissionTime;
     this.queryHandle = new QueryHandle(UUID.randomUUID());
-    this.status = new QueryStatus(0.0f, Status.NEW, "Query just got created", false, null, null);
+    this.status = new QueryStatus(0.0f, Status.NEW, "Query just got created", false, null, null, null);
     this.priority = Priority.NORMAL;
     this.lensConf = qconf;
     this.conf = conf;

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/main/java/org/apache/lens/server/LensServer.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/LensServer.java b/lens-server/src/main/java/org/apache/lens/server/LensServer.java
index e200bdf..d3dfad4 100644
--- a/lens-server/src/main/java/org/apache/lens/server/LensServer.java
+++ b/lens-server/src/main/java/org/apache/lens/server/LensServer.java
@@ -25,7 +25,7 @@ import java.util.logging.Logger;
 
 import javax.ws.rs.core.UriBuilder;
 
-import org.apache.lens.api.response.LensJAXBContextResolver;
+import org.apache.lens.api.result.LensJAXBContextResolver;
 import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.metrics.MetricsService;
 import org.apache.lens.server.error.LensExceptionMapper;

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/main/java/org/apache/lens/server/error/LensExceptionMapper.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/error/LensExceptionMapper.java b/lens-server/src/main/java/org/apache/lens/server/error/LensExceptionMapper.java
index 7240a64..1bb65ca 100644
--- a/lens-server/src/main/java/org/apache/lens/server/error/LensExceptionMapper.java
+++ b/lens-server/src/main/java/org/apache/lens/server/error/LensExceptionMapper.java
@@ -22,7 +22,7 @@ import javax.ws.rs.core.Response;
 import javax.ws.rs.ext.ExceptionMapper;
 import javax.ws.rs.ext.Provider;
 
-import org.apache.lens.api.response.LensResponse;
+import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.server.api.error.LensException;
 
 @Provider
@@ -31,7 +31,7 @@ public class LensExceptionMapper implements ExceptionMapper<LensException> {
   @Override
   public Response toResponse(LensException exception) {
 
-    final LensResponse lensResponse = exception.getLensResponse();
-    return Response.status(lensResponse.getHttpStatusCode()).entity(lensResponse).build();
+    final LensAPIResult lensAPIResult = exception.getLensAPIResult();
+    return Response.status(lensAPIResult.getHttpStatusCode()).entity(lensAPIResult).build();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/main/java/org/apache/lens/server/error/UnSupportedQuerySubmitOpException.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/error/UnSupportedQuerySubmitOpException.java b/lens-server/src/main/java/org/apache/lens/server/error/UnSupportedQuerySubmitOpException.java
index f0a0ab0..4ab26ec 100644
--- a/lens-server/src/main/java/org/apache/lens/server/error/UnSupportedQuerySubmitOpException.java
+++ b/lens-server/src/main/java/org/apache/lens/server/error/UnSupportedQuerySubmitOpException.java
@@ -22,9 +22,9 @@ import static org.apache.lens.server.error.LensServerErrorCode.UNSUPPORTED_QUERY
 
 import org.apache.lens.api.error.ErrorCollection;
 import org.apache.lens.api.error.LensError;
-import org.apache.lens.api.response.LensErrorTO;
+import org.apache.lens.api.query.SupportedQuerySubmitOperations;
+import org.apache.lens.api.result.LensErrorTO;
 import org.apache.lens.server.api.error.LensException;
-import org.apache.lens.server.query.SupportedQuerySubmitOperations;
 
 public class UnSupportedQuerySubmitOpException extends LensException {
 

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/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 e41a0d2..8a1b7c8 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
@@ -35,8 +35,10 @@ import javax.ws.rs.core.StreamingOutput;
 
 import org.apache.lens.api.LensConf;
 import org.apache.lens.api.LensSessionHandle;
+import org.apache.lens.api.error.ErrorCollection;
 import org.apache.lens.api.query.*;
 import org.apache.lens.api.query.QueryStatus.Status;
+import org.apache.lens.api.result.LensErrorTO;
 import org.apache.lens.driver.cube.RewriteUtil;
 import org.apache.lens.driver.hive.HiveDriver;
 import org.apache.lens.server.LensServerConf;
@@ -78,6 +80,8 @@ import org.codehaus.jackson.map.module.SimpleModule;
 import com.google.common.collect.ImmutableList;
 
 import lombok.Getter;
+import lombok.NonNull;
+
 import lombok.extern.slf4j.Slf4j;
 
 /**
@@ -165,7 +169,7 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
   /**
    * The query submitter runnable.
    */
-  private final QuerySubmitter querySubmitterRunnable = new QuerySubmitter();
+  private final QuerySubmitter querySubmitterRunnable = new QuerySubmitter(LensServices.get().getErrorCollection());
 
   /**
    * The query submitter.
@@ -234,6 +238,8 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
 
   private final LogSegregationContext logSegregationContext;
 
+  private final ErrorCollection errorCollection = LensServices.get().getErrorCollection();
+
   /**
    * The driver event listener.
    */
@@ -480,6 +486,11 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
      */
     private boolean pausedForTest = false;
 
+    private final ErrorCollection errorCollection;
+
+    public QuerySubmitter(@NonNull final ErrorCollection errorCollection) {
+      this.errorCollection = errorCollection;
+    }
     /*
      * (non-Javadoc)
      *
@@ -510,10 +521,17 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
                 // Check if we need to pass session's effective resources to selected driver
                 addSessionResourcesToDriver(ctx);
                 ctx.getSelectedDriver().executeAsync(ctx);
+              } catch (LensException e) {
+
+                LOG.error("Error launching query " + ctx.getQueryHandle(), e);
+                String reason = e.getCause() != null ? e.getCause().getMessage() : e.getMessage();
+                setFailedStatus(ctx, "Launching query failed", reason, e.buildLensErrorTO(this.errorCollection));
+                continue;
+
               } catch (Exception e) {
                 log.error("Error launching query " + ctx.getQueryHandle(), e);
                 String reason = e.getCause() != null ? e.getCause().getMessage() : e.getMessage();
-                setFailedStatus(ctx, "Launching query failed", reason);
+                setFailedStatus(ctx, "Launching query failed", reason, null);
                 continue;
               } finally {
                 release(ctx.getLensSessionIdentifier());
@@ -601,9 +619,11 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
    * @param reason    the reason
    * @throws LensException the lens exception
    */
-  void setFailedStatus(QueryContext ctx, String statusMsg, String reason) throws LensException {
+  void setFailedStatus(QueryContext ctx, String statusMsg, String reason, final LensErrorTO lensErrorTO)
+    throws LensException {
+
     QueryStatus before = ctx.getStatus();
-    ctx.setStatus(new QueryStatus(0.0f, QueryStatus.Status.FAILED, statusMsg, false, null, reason));
+    ctx.setStatus(new QueryStatus(0.0f, QueryStatus.Status.FAILED, statusMsg, false, null, reason, lensErrorTO));
     updateFinishedQuery(ctx, before);
     fireStatusChangeEvent(ctx, ctx.getStatus(), before);
   }
@@ -611,7 +631,7 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
   private void setLaunchedStatus(QueryContext ctx) throws LensException {
     QueryStatus before = ctx.getStatus();
     ctx.setStatus(new QueryStatus(ctx.getStatus().getProgress(), QueryStatus.Status.LAUNCHED, "launched on the driver",
-      false, null, null));
+      false, null, null, null));
     launchedQueries.add(ctx);
     ctx.setLaunchTime(System.currentTimeMillis());
     fireStatusChangeEvent(ctx, ctx.getStatus(), before);
@@ -627,7 +647,7 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
    */
   private void setCancelledStatus(QueryContext ctx, String statusMsg) throws LensException {
     QueryStatus before = ctx.getStatus();
-    ctx.setStatus(new QueryStatus(0.0f, QueryStatus.Status.CANCELED, statusMsg, false, null, null));
+    ctx.setStatus(new QueryStatus(0.0f, QueryStatus.Status.CANCELED, statusMsg, false, null, null, null));
     updateFinishedQuery(ctx, before);
     fireStatusChangeEvent(ctx, ctx.getStatus(), before);
   }
@@ -654,7 +674,7 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
   void setSuccessState(QueryContext ctx) throws LensException {
     QueryStatus before = ctx.getStatus();
     ctx.setStatus(new QueryStatus(1.0f, QueryStatus.Status.SUCCESSFUL, "Query is successful!", ctx
-      .isResultAvailableInDriver(), null, null));
+      .isResultAvailableInDriver(), null, null, null));
     updateFinishedQuery(ctx, before);
     fireStatusChangeEvent(ctx, ctx.getStatus(), before);
   }
@@ -678,8 +698,10 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
             ctx.setStatus(ctx.getDriverStatus().toQueryStatus());
           } catch (LensException exc) {
             // Driver gave exception while updating status
-            setFailedStatus(ctx, "Status update failed", exc.getMessage());
+
+            setFailedStatus(ctx, "Status update failed", exc.getMessage(), exc.buildLensErrorTO(this.errorCollection));
             log.error("Status update failed for " + handle, exc);
+
           }
           // query is successfully executed by driver and
           // if query result need not be persisted or there is no result available in driver, move the query to
@@ -831,8 +853,9 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
             resultSets.remove(finished.getCtx().getQueryHandle());
           }
           fireStatusChangeEvent(finished.getCtx(),
-            new QueryStatus(1f, Status.CLOSED, "Query purged", false, null, null), finished.getCtx().getStatus());
+            new QueryStatus(1f, Status.CLOSED, "Query purged", false, null, null, null), finished.getCtx().getStatus());
           log.info("Query purged: " + finished.getCtx().getQueryHandle());
+
         } catch (LensException e) {
           incrCounter(QUERY_PURGER_COUNTER);
           log.error("Error closing  query ", e);
@@ -1546,7 +1569,7 @@ public class QueryExecutionServiceImpl extends LensService implements QueryExecu
   private QueryHandle executeAsyncInternal(LensSessionHandle sessionHandle, QueryContext ctx) throws LensException {
     ctx.setLensSessionIdentifier(sessionHandle.getPublicId().toString());
     QueryStatus before = ctx.getStatus();
-    ctx.setStatus(new QueryStatus(0.0, QueryStatus.Status.QUEUED, "Query is queued", false, null, null));
+    ctx.setStatus(new QueryStatus(0.0, QueryStatus.Status.QUEUED, "Query is queued", false, null, null, null));
     queuedQueries.add(ctx);
     allQueries.put(ctx.getQueryHandle(), ctx);
     fireStatusChangeEvent(ctx, ctx.getStatus(), before);

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/main/java/org/apache/lens/server/query/QueryServiceResource.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/query/QueryServiceResource.java b/lens-server/src/main/java/org/apache/lens/server/query/QueryServiceResource.java
index badde8c..15015d3 100644
--- a/lens-server/src/main/java/org/apache/lens/server/query/QueryServiceResource.java
+++ b/lens-server/src/main/java/org/apache/lens/server/query/QueryServiceResource.java
@@ -33,8 +33,7 @@ import org.apache.lens.api.LensConf;
 import org.apache.lens.api.LensSessionHandle;
 import org.apache.lens.api.error.ErrorCollection;
 import org.apache.lens.api.query.*;
-import org.apache.lens.api.response.LensResponse;
-import org.apache.lens.api.response.NoErrorPayload;
+import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.api.annotations.MultiPurposeResource;
 import org.apache.lens.server.api.error.LensException;
@@ -190,7 +189,7 @@ public class QueryServiceResource {
    * @param timeoutmillis The timeout for the query, honored only in case of value {@value
    *                      SubmitOp#EXECUTE_WITH_TIMEOUT} operation
    * @param queryName     human readable query name set by user (optional parameter)
-   * @return {@link LensResponse} with DATA as {@link QueryHandle} in case of {@value SubmitOp#EXECUTE} operation.
+   * @return {@link LensAPIResult} with DATA as {@link QueryHandle} in case of {@value SubmitOp#EXECUTE} operation.
    * {@link QueryPlan} in case of {@value SubmitOp#EXPLAIN} operation. {@link QueryHandleWithResultSet} in case
    * {@value SubmitOp#EXECUTE_WITH_TIMEOUT} operation. {@link QueryCost} in case of
    * {@value SubmitOp#ESTIMATE} operation.
@@ -200,7 +199,7 @@ public class QueryServiceResource {
   @Consumes({MediaType.MULTIPART_FORM_DATA})
   @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, MediaType.TEXT_PLAIN})
   @MultiPurposeResource(formParamName = "operation")
-  public LensResponse<QuerySubmitResult, NoErrorPayload> query(@FormDataParam("sessionid") LensSessionHandle sessionid,
+  public LensAPIResult<QuerySubmitResult> query(@FormDataParam("sessionid") LensSessionHandle sessionid,
       @FormDataParam("query") String query, @FormDataParam("operation") String operation,
       @FormDataParam("conf") LensConf conf, @DefaultValue("30000") @FormDataParam("timeoutmillis") Long timeoutmillis,
       @DefaultValue("") @FormDataParam("queryName") String queryName) throws LensException {
@@ -230,7 +229,7 @@ public class QueryServiceResource {
         throw new UnSupportedQuerySubmitOpException();
       }
 
-      return LensResponse.composedOf(null, requestId, result);
+      return LensAPIResult.composedOf(null, requestId, result);
     } catch (LensException e) {
       e.buildLensErrorResponse(errorCollection, null, requestId);
       throw e;

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/main/java/org/apache/lens/server/query/ResultFormatter.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/query/ResultFormatter.java b/lens-server/src/main/java/org/apache/lens/server/query/ResultFormatter.java
index de7e120..c526c02 100644
--- a/lens-server/src/main/java/org/apache/lens/server/query/ResultFormatter.java
+++ b/lens-server/src/main/java/org/apache/lens/server/query/ResultFormatter.java
@@ -141,7 +141,7 @@ public class ResultFormatter extends AsyncEventListener<QueryExecuted> {
       metricsService.incrCounter(ResultFormatter.class, "formatting-errors");
       LOG.warn("Exception while formatting result for " + queryHandle, e);
       try {
-        queryService.setFailedStatus(ctx, "Result formatting failed!", e.getMessage());
+        queryService.setFailedStatus(ctx, "Result formatting failed!", e.getMessage(), null);
       } catch (LensException e1) {
         LOG.error("Exception while setting failure for " + queryHandle, e1);
       }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/main/java/org/apache/lens/server/query/SupportedQuerySubmitOperations.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/query/SupportedQuerySubmitOperations.java b/lens-server/src/main/java/org/apache/lens/server/query/SupportedQuerySubmitOperations.java
deleted file mode 100644
index 50858a6..0000000
--- a/lens-server/src/main/java/org/apache/lens/server/query/SupportedQuerySubmitOperations.java
+++ /dev/null
@@ -1,61 +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.query;
-
-import static org.apache.lens.api.query.SubmitOp.*;
-
-import java.util.LinkedList;
-import java.util.List;
-
-import javax.xml.bind.annotation.*;
-
-import org.apache.lens.api.query.SubmitOp;
-
-import org.apache.commons.lang.StringUtils;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-import lombok.EqualsAndHashCode;
-import lombok.ToString;
-
-@XmlRootElement
-@EqualsAndHashCode
-@ToString
-@XmlAccessorType(XmlAccessType.FIELD)
-public class SupportedQuerySubmitOperations {
-
-  private static final String SEP = ", ";
-
-  @XmlElementWrapper(name = "supportedOperations")
-  @XmlElement(name = "operation")
-  private List<String> supportedOps = new LinkedList<String>();
-
-  public SupportedQuerySubmitOperations() {
-    ImmutableSet<SubmitOp> supportedOps = Sets.immutableEnumSet(ESTIMATE, EXECUTE, EXPLAIN, EXECUTE_WITH_TIMEOUT);
-
-    for (SubmitOp supportedOp : supportedOps) {
-      this.supportedOps.add(supportedOp.toString().toLowerCase());
-    }
-  }
-
-  public String getSupportedOperationsAsString() {
-    return StringUtils.join(supportedOps, SEP);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/test/java/org/apache/lens/server/LensTestUtil.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/LensTestUtil.java b/lens-server/src/test/java/org/apache/lens/server/LensTestUtil.java
index c05a2e7..fbf524d 100644
--- a/lens-server/src/test/java/org/apache/lens/server/LensTestUtil.java
+++ b/lens-server/src/test/java/org/apache/lens/server/LensTestUtil.java
@@ -34,8 +34,7 @@ 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.response.LensResponse;
-import org.apache.lens.api.response.NoErrorPayload;
+import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.server.api.LensConfConstants;
 
 import org.apache.commons.io.FileUtils;
@@ -94,7 +93,7 @@ public final class LensTestUtil {
 
     final QueryHandle handle = target.request()
         .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-            new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+            new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     // wait till the query finishes
     LensQuery ctx = target.path(handle.toString()).queryParam("sessionid", lensSessionId).request()
@@ -135,7 +134,7 @@ public final class LensTestUtil {
         MediaType.APPLICATION_XML_TYPE));
 
     final QueryHandle handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     // wait till the query finishes
     LensQuery ctx = target.path(handle.toString()).queryParam("sessionid", lensSessionId).request()
@@ -204,7 +203,7 @@ public final class LensTestUtil {
       MediaType.APPLICATION_XML_TYPE));
 
     final QueryHandle handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     // wait till the query finishes
     LensQuery ctx = target.path(handle.toString()).queryParam("sessionid", lensSessionId).request()

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/test/java/org/apache/lens/server/TestServerMode.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/TestServerMode.java b/lens-server/src/test/java/org/apache/lens/server/TestServerMode.java
index b33b906..28ff920 100644
--- a/lens-server/src/test/java/org/apache/lens/server/TestServerMode.java
+++ b/lens-server/src/test/java/org/apache/lens/server/TestServerMode.java
@@ -35,8 +35,7 @@ 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.response.LensResponse;
-import org.apache.lens.api.response.NoErrorPayload;
+import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.server.LensServices.SERVICE_MODE;
 
 import org.glassfish.jersey.media.multipart.FormDataBodyPart;
@@ -182,7 +181,7 @@ public class TestServerMode extends LensAllApplicationJerseyTest {
     QueryHandle qhandle = null;
     try {
       qhandle = queryTarget.request().post(Entity.entity(query, MediaType.MULTIPART_FORM_DATA_TYPE),
-          new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+          new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
     } catch (NotAllowedException nae) {
       if (mode.equals(SERVICE_MODE.READ_ONLY)) {
         // expected

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java b/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
index 09f8dc7..537e750 100644
--- a/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
+++ b/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
@@ -41,8 +41,7 @@ import org.apache.lens.api.query.LensQuery;
 import org.apache.lens.api.query.PersistentQueryResult;
 import org.apache.lens.api.query.QueryHandle;
 import org.apache.lens.api.query.QueryStatus;
-import org.apache.lens.api.response.LensResponse;
-import org.apache.lens.api.response.NoErrorPayload;
+import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.driver.hive.TestRemoteHiveDriver;
 import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.session.SessionService;
@@ -173,7 +172,7 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
       mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(),
         new LensConf(), MediaType.APPLICATION_XML_TYPE));
       final QueryHandle handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-          new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+          new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
       Assert.assertNotNull(handle);
       LensQuery ctx = target.path(handle.toString()).queryParam("sessionid", lensSessionId).request()
@@ -259,7 +258,7 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
       MediaType.APPLICATION_XML_TYPE));
     QueryHandle handle = target.request()
       .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-          new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+          new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     Assert.assertNotNull(handle);
 
@@ -313,7 +312,7 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
       mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(),
         new LensConf(), MediaType.APPLICATION_XML_TYPE));
       handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-          new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+          new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
       Assert.assertNotNull(handle);
 
       // Poll for second query, this should finish successfully

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/test/java/org/apache/lens/server/common/ErrorResponseExpectedData.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/common/ErrorResponseExpectedData.java b/lens-server/src/test/java/org/apache/lens/server/common/ErrorResponseExpectedData.java
index 3963a06..38b5ea0 100644
--- a/lens-server/src/test/java/org/apache/lens/server/common/ErrorResponseExpectedData.java
+++ b/lens-server/src/test/java/org/apache/lens/server/common/ErrorResponseExpectedData.java
@@ -24,8 +24,9 @@ import static org.testng.Assert.assertTrue;
 
 import javax.ws.rs.core.Response;
 
-import org.apache.lens.api.response.LensErrorTO;
-import org.apache.lens.api.response.LensResponse;
+import org.apache.lens.api.result.LensAPIResult;
+import org.apache.lens.api.result.LensErrorTO;
+
 
 public class ErrorResponseExpectedData {
 
@@ -44,14 +45,14 @@ public class ErrorResponseExpectedData {
     /* Assert Equal Http Status Code */
     assertEquals(response.getStatus(), expectedStatus.getStatusCode());
 
-    LensResponse lensResponse = response.readEntity(LensResponse.class);
+    LensAPIResult lensAPIResult = response.readEntity(LensAPIResult.class);
 
     /* Assert Equal LensErrorTO (stack trace gets excluded in equality check) */
-    final LensErrorTO actualLensErrorTO = lensResponse.getLensErrorTO();
+    final LensErrorTO actualLensErrorTO = lensAPIResult.getLensErrorTO();
     assertEquals(actualLensErrorTO, expectedLensErrorTO);
 
     /* Assert receipt of valid stacktraces */
-    assertTrue(lensResponse.areValidStackTracesPresent(), "Received Lens Response:" + lensResponse);
+    assertTrue(lensAPIResult.areValidStackTracesPresent(), "Received Lens Response:" + lensAPIResult);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/test/java/org/apache/lens/server/common/TestDataUtils.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/common/TestDataUtils.java b/lens-server/src/test/java/org/apache/lens/server/common/TestDataUtils.java
index a59252c..0400519 100644
--- a/lens-server/src/test/java/org/apache/lens/server/common/TestDataUtils.java
+++ b/lens-server/src/test/java/org/apache/lens/server/common/TestDataUtils.java
@@ -62,21 +62,16 @@ public class TestDataUtils {
     return "dimfield" + getRandomName();
   }
 
-  public static XCube createXCube(final String cubeName, final XDimAttribute... dimensions) {
-
-    return createXCube(cubeName, Optional.<String>absent(), dimensions);
-  }
-
-  public static XCube createXCube(final String cubeName, final Optional<String> dtColName,
+  public static XCube createXCubeWithDummyMeasure(final String cubeName, final Optional<String> dtColName,
       final XDimAttribute... dimensions) {
 
     XDimAttributes xDimAttributes = createXDimAttributes(dimensions);
     XMeasures xMeasures = createXMeasures(createDummyXMeasure());
 
-    return createXCube(cubeName, dtColName, xDimAttributes, xMeasures);
+    return createXCubeWithDummyMeasure(cubeName, dtColName, xDimAttributes, xMeasures);
   }
 
-  public static XCube createXCube(final String cubeName, final Optional<String> dtColName,
+  public static XCube createXCubeWithDummyMeasure(final String cubeName, final Optional<String> dateColName,
       final XDimAttributes xDimAttributes, final XMeasures xMeasures) {
 
     XBaseCube cube = new XBaseCube();
@@ -84,11 +79,11 @@ public class TestDataUtils {
     cube.setDimAttributes(xDimAttributes);
     cube.setMeasures(xMeasures);
 
-    if (dtColName.isPresent()) {
+    if (dateColName.isPresent()) {
 
       XProperty xp = new XProperty();
       xp.setName("cube."+cubeName+".timed.dimensions.list");
-      xp.setValue(dtColName.get());
+      xp.setValue(dateColName.get());
 
       XProperties xProperties = new XProperties();
       xProperties.getProperty().add(xp);

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/test/java/org/apache/lens/server/metrics/TestResourceMethodMetrics.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/metrics/TestResourceMethodMetrics.java b/lens-server/src/test/java/org/apache/lens/server/metrics/TestResourceMethodMetrics.java
index 695403e..118970d 100644
--- a/lens-server/src/test/java/org/apache/lens/server/metrics/TestResourceMethodMetrics.java
+++ b/lens-server/src/test/java/org/apache/lens/server/metrics/TestResourceMethodMetrics.java
@@ -38,8 +38,7 @@ import org.apache.lens.api.APIResult;
 import org.apache.lens.api.LensConf;
 import org.apache.lens.api.LensSessionHandle;
 import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.response.LensResponse;
-import org.apache.lens.api.response.NoErrorPayload;
+import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.server.LensAllApplicationJerseyTest;
 import org.apache.lens.server.LensApplication;
 import org.apache.lens.server.LensServices;
@@ -242,7 +241,7 @@ public class TestResourceMethodMetrics extends LensAllApplicationJerseyTest {
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
       MediaType.APPLICATION_XML_TYPE));
     final QueryHandle handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     Assert.assertNotNull(handle);
   }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/test/java/org/apache/lens/server/query/QueryAPIErrorResponseTest.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/QueryAPIErrorResponseTest.java b/lens-server/src/test/java/org/apache/lens/server/query/QueryAPIErrorResponseTest.java
index 720229d..91e9081 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/QueryAPIErrorResponseTest.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/QueryAPIErrorResponseTest.java
@@ -41,8 +41,9 @@ import org.apache.lens.api.metastore.XColumn;
 import org.apache.lens.api.metastore.XCube;
 import org.apache.lens.api.metastore.XDimAttribute;
 import org.apache.lens.api.metastore.XFactTable;
-import org.apache.lens.api.response.LensErrorTO;
-import org.apache.lens.api.response.LensJAXBContextResolver;
+import org.apache.lens.api.query.SupportedQuerySubmitOperations;
+import org.apache.lens.api.result.LensErrorTO;
+import org.apache.lens.api.result.LensJAXBContextResolver;
 import org.apache.lens.cube.error.ColUnAvailableInTimeRange;
 import org.apache.lens.server.LensJerseyTest;
 import org.apache.lens.server.LensRequestContextInitFilter;
@@ -264,13 +265,13 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
 
       /* Create a test cube with test dimension field having a start Date and end Date */
       XDimAttribute testXDim = createXDimAttribute(testDimensionField, colStartDate, colEndDate);
-      XCube xcube = createXCube(testCube, Optional.of("dt"), testXDim);
+      XCube xcube = createXCubeWithDummyMeasure(testCube, Optional.of("dt"), testXDim);
       createCubeFailFast(target, sessionId, xcube);
 
       /* Create a fact with test dimension field */
       XColumn xColumn = createXColumn(testDimensionField);
       XFactTable xFactTable = createXFactTableWithColumns(testFact, testCube, xColumn);
-      createFact(target, sessionId, xFactTable);
+      createFactFailFast(target, sessionId, xFactTable);
 
       /* Setup: End */
 

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/test/java/org/apache/lens/server/query/TestQueryEndEmailNotifier.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryEndEmailNotifier.java b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryEndEmailNotifier.java
index d3fa02a..0b152bf 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryEndEmailNotifier.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryEndEmailNotifier.java
@@ -35,8 +35,7 @@ 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.api.response.LensResponse;
-import org.apache.lens.api.response.NoErrorPayload;
+import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.server.LensJerseyTest;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.LensTestUtil;
@@ -176,7 +175,7 @@ public class TestQueryEndEmailNotifier extends LensJerseyTest {
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
       MediaType.APPLICATION_XML_TYPE));
     final QueryHandle handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-      new GenericType<LensResponse<QueryHandle, NoErrorPayload>>(){}).getData();
+      new GenericType<LensAPIResult<QueryHandle>>(){}).getData();
 
     Assert.assertNotNull(handle);
     LensQuery ctx = target.path(handle.toString()).queryParam("sessionid", lensSessionId).request()

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
index 62c24ba..655549b 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/TestQueryService.java
@@ -39,10 +39,9 @@ import org.apache.lens.api.LensSessionHandle;
 import org.apache.lens.api.error.LensCommonErrorCode;
 import org.apache.lens.api.query.*;
 import org.apache.lens.api.query.QueryStatus.Status;
-import org.apache.lens.api.response.LensErrorTO;
-import org.apache.lens.api.response.LensJAXBContextResolver;
-import org.apache.lens.api.response.LensResponse;
-import org.apache.lens.api.response.NoErrorPayload;
+import org.apache.lens.api.result.LensAPIResult;
+import org.apache.lens.api.result.LensErrorTO;
+import org.apache.lens.api.result.LensJAXBContextResolver;
 import org.apache.lens.driver.hive.HiveDriver;
 import org.apache.lens.driver.hive.TestHiveDriver;
 import org.apache.lens.server.LensJerseyTest;
@@ -236,7 +235,7 @@ public class TestQueryService extends LensJerseyTest {
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
       MediaType.APPLICATION_XML_TYPE));
     final QueryHandle handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {
+        new GenericType<LensAPIResult<QueryHandle>>() {
         }).getData();
 
     Assert.assertNotNull(handle);
@@ -287,7 +286,7 @@ public class TestQueryService extends LensJerseyTest {
     long finishedQueries = metricsSvc.getFinishedQueries();
 
     final QueryHandle handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {
+        new GenericType<LensAPIResult<QueryHandle>>() {
         }).getData();
 
     Assert.assertNotNull(handle);
@@ -379,7 +378,7 @@ public class TestQueryService extends LensJerseyTest {
 
     final QueryPlan plan = target.request()
       .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-          new GenericType<LensResponse<QueryPlan, NoErrorPayload>>() {}).getData();
+          new GenericType<LensAPIResult<QueryPlan>>() {}).getData();
     Assert.assertEquals(plan.getTablesQueried().size(), 1);
     Assert.assertTrue(plan.getTablesQueried().get(0).endsWith(TEST_TABLE.toLowerCase()));
     Assert.assertNull(plan.getPrepareHandle());
@@ -425,7 +424,7 @@ public class TestQueryService extends LensJerseyTest {
 
     final QueryPlan plan = target.request()
       .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-          new GenericType<LensResponse<QueryPlan, NoErrorPayload>>() {}).getData();
+          new GenericType<LensAPIResult<QueryPlan>>() {}).getData();
     Assert.assertTrue(plan.isError());
     Assert.assertNotNull(plan.getErrorMsg());
     Assert.assertTrue(plan.getErrorMsg()
@@ -676,7 +675,7 @@ public class TestQueryService extends LensJerseyTest {
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
       MediaType.APPLICATION_XML_TYPE));
     final QueryHandle handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     Assert.assertNotNull(handle);
 
@@ -715,7 +714,7 @@ public class TestQueryService extends LensJerseyTest {
 
     // test cancel query
     final QueryHandle handle2 = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     Assert.assertNotNull(handle2);
     APIResult result = target.path(handle2.toString()).queryParam("sessionid", lensSessionId).request()
@@ -747,7 +746,7 @@ public class TestQueryService extends LensJerseyTest {
     mp3.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
       MediaType.APPLICATION_XML_TYPE));
     final QueryHandle handle3 = target.request().post(Entity.entity(mp3, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     // Get query
     ctx = target.path(handle3.toString()).queryParam("sessionid", lensSessionId).request().get(LensQuery.class);
@@ -961,7 +960,7 @@ public class TestQueryService extends LensJerseyTest {
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
       MediaType.APPLICATION_XML_TYPE));
     final QueryHandle handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     Assert.assertNotNull(handle);
 
@@ -1016,7 +1015,7 @@ public class TestQueryService extends LensJerseyTest {
     drop.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
       MediaType.APPLICATION_XML_TYPE));
     final QueryHandle dropHandle = target.request().post(Entity.entity(drop, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     Assert.assertNotNull(dropHandle);
 
@@ -1047,7 +1046,7 @@ public class TestQueryService extends LensJerseyTest {
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
       MediaType.APPLICATION_XML_TYPE));
     final QueryHandle handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     Assert.assertNotNull(handle);
 
@@ -1075,7 +1074,7 @@ public class TestQueryService extends LensJerseyTest {
     fetch.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), conf,
       MediaType.APPLICATION_XML_TYPE));
     final QueryHandle handle2 = target.request().post(Entity.entity(fetch, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     Assert.assertNotNull(handle2);
 
@@ -1184,7 +1183,7 @@ public class TestQueryService extends LensJerseyTest {
       MediaType.APPLICATION_XML_TYPE));
 
     QueryHandleWithResultSet result = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandleWithResultSet, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandleWithResultSet>>() {}).getData();
     Assert.assertNotNull(result.getQueryHandle());
     Assert.assertNotNull(result.getResult());
     validatePersistentResult((PersistentQueryResult) result.getResult(), result.getQueryHandle(), true);
@@ -1203,7 +1202,7 @@ public class TestQueryService extends LensJerseyTest {
       MediaType.APPLICATION_XML_TYPE));
 
     result = target.request().post(Entity.entity(mp2, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandleWithResultSet, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandleWithResultSet>>() {}).getData();
     Assert.assertNotNull(result.getQueryHandle());
     Assert.assertNotNull(result.getResult());
     validateInmemoryResult((InMemoryQueryResult) result.getResult());
@@ -1230,7 +1229,7 @@ public class TestQueryService extends LensJerseyTest {
       MediaType.APPLICATION_XML_TYPE));
 
     QueryHandleWithResultSet result = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensResponse<QueryHandleWithResultSet, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryHandleWithResultSet>>() {}).getData();
     Assert.assertNotNull(result.getQueryHandle());
     Assert.assertNull(result.getResult());
 
@@ -1322,7 +1321,7 @@ public class TestQueryService extends LensJerseyTest {
 
     final QueryCost result = target.request()
       .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-          new GenericType<LensResponse<QueryCost, NoErrorPayload>>() {}).getData();
+          new GenericType<LensAPIResult<QueryCost>>() {}).getData();
     Assert.assertNotNull(result);
     Assert.assertEquals(result.getEstimatedExecTimeMillis(), 1L);
     Assert.assertEquals(result.getEstimatedResourceUsage(), 1.0);
@@ -1477,7 +1476,7 @@ public class TestQueryService extends LensJerseyTest {
 
     final QueryCost queryCost = target.request()
       .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-          new GenericType<LensResponse<QueryCost, NoErrorPayload>>() {
+          new GenericType<LensAPIResult<QueryCost>>() {
           }).getData();
     Assert.assertNotNull(queryCost);
 

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-server/src/test/java/org/apache/lens/server/query/TestResultFormatting.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/TestResultFormatting.java b/lens-server/src/test/java/org/apache/lens/server/query/TestResultFormatting.java
index f65805d..ea81e90 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/TestResultFormatting.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/TestResultFormatting.java
@@ -35,8 +35,7 @@ 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.api.response.LensResponse;
-import org.apache.lens.api.response.NoErrorPayload;
+import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.server.LensJerseyTest;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.LensTestUtil;
@@ -214,7 +213,7 @@ public class TestResultFormatting extends LensJerseyTest {
       MediaType.APPLICATION_XML_TYPE));
     QueryHandle handle = target.request()
       .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-          new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+          new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
     Assert.assertNotNull(handle);
 

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8744974..a896aea 100644
--- a/pom.xml
+++ b/pom.xml
@@ -81,7 +81,7 @@
 
     <!-- test -->
     <testng.version>6.8</testng.version>
-    <mockito.version>1.8.5</mockito.version>
+    <mockito.version>1.9.5</mockito.version>
     <powermock.version>1.4.12</powermock.version>
     <test.tmp.dir>${basedir}/target/tmp</test.tmp.dir>
 


[2/2] incubator-lens git commit: LENS-487: Implementation for consistent error response display for Lens Client

Posted by hi...@apache.org.
LENS-487: Implementation for consistent error response display for Lens Client


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

Branch: refs/heads/master
Commit: d1d99708026d17d43f3f9ea4f9ae088cb151fbd2
Parents: 5d20751
Author: Himanshu Gahlaut <ga...@gmail.com>
Authored: Thu Jun 11 11:49:37 2015 +0530
Committer: Himanshu Gahlaut <ga...@gmail.com>
Committed: Sat Jun 20 14:23:53 2015 +0530

----------------------------------------------------------------------
 .../org/apache/lens/api/query/LensQuery.java    |  11 ++
 .../org/apache/lens/api/query/QueryStatus.java  |  12 ++
 .../query/SupportedQuerySubmitOperations.java   |  59 +++++++++
 .../apache/lens/api/response/LensErrorTO.java   | 119 ------------------
 .../api/response/LensJAXBContextResolver.java   |  77 ------------
 .../apache/lens/api/response/LensResponse.java  | 110 ----------------
 .../lens/api/response/NoErrorPayload.java       |  33 -----
 .../api/response/NoSuccessResponseData.java     |  30 -----
 .../apache/lens/api/result/LensAPIResult.java   | 125 +++++++++++++++++++
 .../org/apache/lens/api/result/LensErrorTO.java | 122 ++++++++++++++++++
 .../api/result/LensJAXBContextResolver.java     |  77 ++++++++++++
 .../apache/lens/api/result/NoErrorPayload.java  |  33 +++++
 .../apache/lens/api/result/NoResultData.java    |  30 +++++
 .../apache/lens/api/result/PrettyPrintable.java |  32 +++++
 lens-api/src/main/resources/lens-errors.conf    |   2 +-
 .../lens/cli/commands/LensQueryCommands.java    |  33 +++--
 .../apache/lens/cli/ExecuteQueryCommandIT.java  |  42 +++++++
 .../apache/lens/cli/TestLensQueryCommands.java  |   3 +-
 lens-client/pom.xml                             |  20 +++
 .../java/org/apache/lens/client/LensClient.java |  19 ++-
 .../org/apache/lens/client/LensStatement.java   |  64 ++++++----
 .../client/exceptions/LensAPIException.java     |  48 +++++++
 .../exceptions/LensBriefErrorException.java     |  34 +++++
 .../client/exceptions/LensClientException.java  |  24 +---
 .../lens/client/jdbc/LensJdbcStatement.java     |  10 +-
 .../apache/lens/client/model/BriefError.java    |  47 +++++++
 .../lens/client/model/IdBriefErrorTemplate.java |  53 ++++++++
 .../client/model/IdBriefErrorTemplateKey.java   |  35 ++++++
 .../lens/client/model/BriefErrorTest.java       |  60 +++++++++
 .../client/model/IdBriefErrorTemplateTest.java  |  74 +++++++++++
 .../ColUnAvailableInTimeRangeException.java     |   2 +-
 .../FieldsCannotBeQueriedTogetherException.java |   2 +-
 .../org/apache/lens/examples/SampleQueries.java |   7 +-
 .../org/apache/lens/ml/impl/LensMLImpl.java     |   5 +-
 .../java/org/apache/lens/rdd/LensRDDClient.java |   7 +-
 .../regression/core/helpers/QueryHelper.java    |  12 +-
 .../server/api/driver/DriverQueryStatus.java    |   4 +-
 .../lens/server/api/error/LensException.java    |   8 +-
 .../api/error/LensMultiCauseException.java      |   2 +-
 .../server/api/query/FinishedLensQuery.java     |   2 +-
 .../lens/server/api/query/QueryContext.java     |   2 +-
 .../java/org/apache/lens/server/LensServer.java |   2 +-
 .../lens/server/error/LensExceptionMapper.java  |   6 +-
 .../UnSupportedQuerySubmitOpException.java      |   4 +-
 .../server/query/QueryExecutionServiceImpl.java |  43 +++++--
 .../lens/server/query/QueryServiceResource.java |   9 +-
 .../lens/server/query/ResultFormatter.java      |   2 +-
 .../query/SupportedQuerySubmitOperations.java   |  61 ---------
 .../org/apache/lens/server/LensTestUtil.java    |   9 +-
 .../org/apache/lens/server/TestServerMode.java  |   5 +-
 .../apache/lens/server/TestServerRestart.java   |   9 +-
 .../common/ErrorResponseExpectedData.java       |  11 +-
 .../lens/server/common/TestDataUtils.java       |  15 +--
 .../metrics/TestResourceMethodMetrics.java      |   5 +-
 .../server/query/QueryAPIErrorResponseTest.java |   9 +-
 .../server/query/TestQueryEndEmailNotifier.java |   5 +-
 .../lens/server/query/TestQueryService.java     |  39 +++---
 .../lens/server/query/TestResultFormatting.java |   5 +-
 pom.xml                                         |   2 +-
 59 files changed, 1131 insertions(+), 601 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/query/LensQuery.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/query/LensQuery.java b/lens-api/src/main/java/org/apache/lens/api/query/LensQuery.java
index c22d39a..f594133 100644
--- a/lens-api/src/main/java/org/apache/lens/api/query/LensQuery.java
+++ b/lens-api/src/main/java/org/apache/lens/api/query/LensQuery.java
@@ -209,4 +209,15 @@ public class LensQuery {
   @Getter
   private String queryName;
 
+  public Integer getErrorCode() {
+    return (this.status != null) ? this.status.getErrorCode() : null;
+  }
+
+  public String getErrorMessage() {
+    return (this.status!=null) ? this.status.getLensErrorTOErrorMsg() : null;
+  }
+
+  public String getQueryHandleString() {
+    return (this.queryHandle != null) ? this.queryHandle.getHandleIdString() : null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/query/QueryStatus.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/query/QueryStatus.java b/lens-api/src/main/java/org/apache/lens/api/query/QueryStatus.java
index f927375..539f0b0 100644
--- a/lens-api/src/main/java/org/apache/lens/api/query/QueryStatus.java
+++ b/lens-api/src/main/java/org/apache/lens/api/query/QueryStatus.java
@@ -26,6 +26,8 @@ import java.io.Serializable;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.lens.api.result.LensErrorTO;
+
 import lombok.*;
 
 /**
@@ -153,6 +155,9 @@ public class QueryStatus implements Serializable {
   @Setter
   private String errorMessage;
 
+  @XmlElement
+  private LensErrorTO lensErrorTO;
+
   /*
    * (non-Javadoc)
    *
@@ -253,4 +258,11 @@ public class QueryStatus implements Serializable {
     return isValidTransition(this.status, newState);
   }
 
+  public Integer getErrorCode() {
+    return (this.lensErrorTO != null) ? this.lensErrorTO.getCode() : null;
+  }
+
+  public String getLensErrorTOErrorMsg() {
+    return (this.lensErrorTO != null) ? this.lensErrorTO.getMessage() : null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/query/SupportedQuerySubmitOperations.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/query/SupportedQuerySubmitOperations.java b/lens-api/src/main/java/org/apache/lens/api/query/SupportedQuerySubmitOperations.java
new file mode 100644
index 0000000..5f246e5
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/query/SupportedQuerySubmitOperations.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.api.query;
+
+import static org.apache.lens.api.query.SubmitOp.*;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import javax.xml.bind.annotation.*;
+
+import org.apache.commons.lang.StringUtils;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+
+@XmlRootElement
+@EqualsAndHashCode
+@ToString
+@XmlAccessorType(XmlAccessType.FIELD)
+public class SupportedQuerySubmitOperations {
+
+  private static final String SEP = ", ";
+
+  @XmlElementWrapper(name = "supportedOperations")
+  @XmlElement(name = "operation")
+  private List<String> supportedOps = new LinkedList<String>();
+
+  public SupportedQuerySubmitOperations() {
+    ImmutableSet<SubmitOp> supportedOps = Sets.immutableEnumSet(ESTIMATE, EXECUTE, EXPLAIN, EXECUTE_WITH_TIMEOUT);
+
+    for (SubmitOp supportedOp : supportedOps) {
+      this.supportedOps.add(supportedOp.toString().toLowerCase());
+    }
+  }
+
+  public String getSupportedOperationsAsString() {
+    return StringUtils.join(supportedOps, SEP);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/response/LensErrorTO.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/response/LensErrorTO.java b/lens-api/src/main/java/org/apache/lens/api/response/LensErrorTO.java
deleted file mode 100644
index 313566c..0000000
--- a/lens-api/src/main/java/org/apache/lens/api/response/LensErrorTO.java
+++ /dev/null
@@ -1,119 +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.api.response;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.util.List;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlElement;
-import javax.xml.bind.annotation.XmlElementWrapper;
-
-import org.apache.commons.lang.StringUtils;
-
-import lombok.*;
-
-/**
- *
- * Transport object for lens error information.
- *
- * @param <PAYLOAD>  represents type of error payload transferred in failure lens response
- */
-@EqualsAndHashCode(exclude = { "stackTrace" })
-@ToString
-@NoArgsConstructor(access =  AccessLevel.PACKAGE)
-@XmlAccessorType(XmlAccessType.FIELD)
-public class LensErrorTO<PAYLOAD> {
-
-  @XmlElement
-  private int code;
-
-  @XmlElement
-  private String message;
-
-  @XmlElement
-  private String stackTrace;
-
-  @XmlElement
-  @Getter
-  private PAYLOAD payload;
-
-  @Getter
-  @XmlElementWrapper(name = "childErrors")
-  @XmlElement(name = "error")
-  private List<LensErrorTO> childErrors;
-
-  public static <PAYLOAD> LensErrorTO<PAYLOAD> composedOf(final int code, final String message,
-      final String stackTrace, final PAYLOAD payload, final List<LensErrorTO> childErrors) {
-
-    return new LensErrorTO<PAYLOAD>(code, message, stackTrace, payload, childErrors);
-  }
-
-  public static <PAYLOAD> LensErrorTO<PAYLOAD> composedOf(final int code, final String message,
-      final String stackTrace, final PAYLOAD payload) {
-
-    return new LensErrorTO<PAYLOAD>(code, message, stackTrace, payload, null);
-  }
-
-  public static LensErrorTO<NoErrorPayload> composedOf(final int code, final String message,
-      final String stackTrace) {
-
-    return new LensErrorTO<NoErrorPayload>(code, message, stackTrace, null, null);
-  }
-
-  public static LensErrorTO<NoErrorPayload> composedOf(final int code, final String message,
-      final String stackTrace, final List<LensErrorTO> childErrors) {
-
-    return new LensErrorTO<NoErrorPayload>(code, message, stackTrace, null, childErrors);
-  }
-
-  private LensErrorTO(final int code, final String message, final String stackTrace, final PAYLOAD errorPayload,
-      final List<LensErrorTO> childErrors) {
-
-    checkArgument(code > 0);
-    checkArgument(StringUtils.isNotBlank(message));
-    checkArgument(StringUtils.isNotBlank(stackTrace));
-
-    this.code = code;
-    this.message = message;
-    this.stackTrace = stackTrace;
-    this.payload = errorPayload;
-    this.childErrors = childErrors;
-  }
-
-  public boolean areValidStackTracesPresent() {
-
-    /* if stack trace of first level error is not valid, then return false */
-    if (StringUtils.isBlank(stackTrace)) {
-      return false;
-    }
-
-    /* validate stack traces of child Errors */
-    if (childErrors != null) {
-      for (LensErrorTO childError : childErrors) {
-        if (!childError.areValidStackTracesPresent()) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/response/LensJAXBContextResolver.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/response/LensJAXBContextResolver.java b/lens-api/src/main/java/org/apache/lens/api/response/LensJAXBContextResolver.java
deleted file mode 100644
index a401b53..0000000
--- a/lens-api/src/main/java/org/apache/lens/api/response/LensJAXBContextResolver.java
+++ /dev/null
@@ -1,77 +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.api.response;
-
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import javax.ws.rs.ext.ContextResolver;
-import javax.ws.rs.ext.Provider;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-
-import org.apache.lens.api.error.ErrorCollection;
-import org.apache.lens.api.error.ErrorCollectionFactory;
-
-import com.google.common.collect.Sets;
-import lombok.extern.slf4j.Slf4j;
-
-/**
- * @see javax.ws.rs.ext.ContextResolver
- */
-@Provider
-@Slf4j
-public class LensJAXBContextResolver implements ContextResolver<JAXBContext> {
-
-  private Map<Class, JAXBContext> jaxbContextCache = new ConcurrentHashMap<Class, JAXBContext>();
-
-  @Override
-  public JAXBContext getContext(Class<?> type) {
-
-    JAXBContext jaxbContext = jaxbContextCache.get(type);
-
-    if (jaxbContext == null) {
-
-      log.debug("JAXB instance to be created for {}", type);
-      try {
-        if (type.equals(LensResponse.class)) {
-
-          ErrorCollection errorCollection = new ErrorCollectionFactory().createErrorCollection();
-          Set<Class> classesToBeBound = Sets.newHashSet(errorCollection.getErrorPayloadClasses());
-          log.debug("classesToBeBound:{}", classesToBeBound);
-          classesToBeBound.add(type);
-
-          Class[] classesToBeBoundArray = classesToBeBound.toArray(new Class[classesToBeBound.size()]);
-          jaxbContext = JAXBContext.newInstance(classesToBeBoundArray);
-        } else {
-
-          jaxbContext = JAXBContext.newInstance(type);
-        }
-        jaxbContextCache.put(type, jaxbContext);
-
-      } catch (JAXBException e) {
-        log.error("JAXBContext not initialized for "+type, e);
-      } catch (ClassNotFoundException e) {
-        log.error("JAXBContext not initialized for "+type, e);
-      }
-    }
-    return jaxbContext;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/response/LensResponse.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/response/LensResponse.java b/lens-api/src/main/java/org/apache/lens/api/response/LensResponse.java
deleted file mode 100644
index 818ae40..0000000
--- a/lens-api/src/main/java/org/apache/lens/api/response/LensResponse.java
+++ /dev/null
@@ -1,110 +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.api.response;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import javax.ws.rs.core.Response.Status;
-import javax.xml.bind.annotation.*;
-
-import org.apache.lens.api.query.QuerySubmitResult;
-
-import org.apache.commons.lang.StringUtils;
-
-import lombok.*;
-
-/**
- * Transport object for LensResponse
- *
- * DATA represents type of data transferred in success response.
- * PAYLOAD represents type of error payload transferred in error response.
- *
- */
-@XmlRootElement
-@XmlSeeAlso({NoSuccessResponseData.class, NoErrorPayload.class, QuerySubmitResult.class})
-@NoArgsConstructor(access=AccessLevel.PACKAGE)
-@ToString
-@XmlAccessorType(XmlAccessType.FIELD)
-public class LensResponse<DATA, PAYLOAD> {
-
-  @XmlElement
-  private String apiVersion;
-
-  @XmlElement
-  private String id;
-
-  @XmlElement(name = "data")
-  @Getter
-  private DATA data;
-
-  @XmlElement(name = "error")
-  @Getter
-  private LensErrorTO<PAYLOAD> lensErrorTO;
-
-  @XmlTransient
-  private Status httpStatusCode;
-
-  public static <DATA> LensResponse<DATA, NoErrorPayload> composedOf(final String apiVersion,
-      final String id, @NonNull final DATA data) {
-
-    return new LensResponse<DATA, NoErrorPayload>(apiVersion, id, data, null, Status.OK);
-  }
-
-  public static <DATA> LensResponse<DATA, NoErrorPayload> composedOf(final String apiVersion,
-      final String id, @NonNull final DATA data, @NonNull final Status httpStatusCode) {
-
-    return new LensResponse<DATA, NoErrorPayload>(apiVersion, id, data, null, httpStatusCode);
-  }
-
-  public static <PAYLOAD> LensResponse<NoSuccessResponseData, PAYLOAD> composedOf(
-      final String apiVersion, final String id, @NonNull final LensErrorTO<PAYLOAD> lensErrorTO,
-      @NonNull final Status httpStatusCode) {
-
-    return new LensResponse<NoSuccessResponseData, PAYLOAD>(apiVersion, id, null, lensErrorTO, httpStatusCode);
-  }
-
-  private LensResponse(final String apiVersion, final String id, final DATA data,
-      final LensErrorTO lensErrorTO, @NonNull final Status httpStatusCode) {
-
-    /* The check commented below should be enabled in future, once story of apiVersion is clear. Right now there could
-    be REST APIs throwing LensException without initializing apiVersion
-
-    checkArgument(StringUtils.isNotBlank(apiVersion)); */
-
-    checkArgument(StringUtils.isNotBlank(id));
-
-    this.apiVersion = apiVersion;
-    this.id = id;
-    this.data = data;
-    this.lensErrorTO = lensErrorTO;
-    this.httpStatusCode = httpStatusCode;
-  }
-
-  public boolean areValidStackTracesPresent() {
-    return (lensErrorTO == null) ? false : lensErrorTO.areValidStackTracesPresent();
-  }
-
-  public Status getHttpStatusCode() {
-    return this.httpStatusCode;
-  }
-
-  public boolean isSuccessResponse() {
-    return lensErrorTO == null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/response/NoErrorPayload.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/response/NoErrorPayload.java b/lens-api/src/main/java/org/apache/lens/api/response/NoErrorPayload.java
deleted file mode 100644
index afc1a05..0000000
--- a/lens-api/src/main/java/org/apache/lens/api/response/NoErrorPayload.java
+++ /dev/null
@@ -1,33 +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.api.response;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-import lombok.EqualsAndHashCode;
-
-/**
- * NoErrorPayload type is to be used while creating LensResponse for success responses.
- * Success Responses will not have any error payload.
- *
- */
-@XmlRootElement
-@EqualsAndHashCode
-public final class NoErrorPayload {
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/response/NoSuccessResponseData.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/response/NoSuccessResponseData.java b/lens-api/src/main/java/org/apache/lens/api/response/NoSuccessResponseData.java
deleted file mode 100644
index 1775584..0000000
--- a/lens-api/src/main/java/org/apache/lens/api/response/NoSuccessResponseData.java
+++ /dev/null
@@ -1,30 +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.api.response;
-
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * NoSuccessResponseData type is to be used while creating LensResponse for error responses.
- * Error Responses will not have any data related to successful execution of API.
- *
- */
-@XmlRootElement
-public final class NoSuccessResponseData {
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/result/LensAPIResult.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/result/LensAPIResult.java b/lens-api/src/main/java/org/apache/lens/api/result/LensAPIResult.java
new file mode 100644
index 0000000..41805eb
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/result/LensAPIResult.java
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.api.result;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+import javax.ws.rs.core.Response.Status;
+import javax.xml.bind.annotation.*;
+
+import org.apache.lens.api.query.QuerySubmitResult;
+
+import org.apache.commons.lang.StringUtils;
+
+import lombok.*;
+
+/**
+ * Transport object for results returned by Lens APIs
+ *
+ * DATA represents type of data in success result.
+ *
+ */
+@XmlRootElement
+@XmlSeeAlso({NoResultData.class, NoErrorPayload.class, QuerySubmitResult.class})
+@NoArgsConstructor(access=AccessLevel.PACKAGE)
+@ToString
+@XmlAccessorType(XmlAccessType.FIELD)
+public class LensAPIResult<DATA> {
+
+  @XmlElement
+  private String apiVersion;
+
+  @XmlElement
+  @Getter
+  private String id;
+
+  @XmlElement(name = "data")
+  @Getter
+  private DATA data;
+
+  @XmlElement(name = "error")
+  @Getter
+  private LensErrorTO lensErrorTO;
+
+  @XmlTransient
+  private Status httpStatusCode;
+
+  public static <DATA> LensAPIResult<DATA> composedOf(final String apiVersion,
+      final String id, @NonNull final DATA data) {
+
+    return new LensAPIResult<DATA>(apiVersion, id, data, null, Status.OK);
+  }
+
+  public static <DATA> LensAPIResult<DATA> composedOf(final String apiVersion,
+      final String id, @NonNull final DATA data, @NonNull final Status httpStatusCode) {
+
+    return new LensAPIResult<DATA>(apiVersion, id, data, null, httpStatusCode);
+  }
+
+  public static <PAYLOAD> LensAPIResult<NoResultData> composedOf(
+      final String apiVersion, final String id, @NonNull final LensErrorTO lensErrorTO,
+      @NonNull final Status httpStatusCode) {
+
+    return new LensAPIResult<NoResultData>(apiVersion, id, null, lensErrorTO, httpStatusCode);
+  }
+
+  private LensAPIResult(final String apiVersion, final String id, final DATA data, final LensErrorTO lensErrorTO,
+      @NonNull final Status httpStatusCode) {
+
+    /* The check commented below should be enabled in future, once story of apiVersion is clear. Right now there could
+    be REST APIs throwing LensException without initializing apiVersion
+
+    checkArgument(StringUtils.isNotBlank(apiVersion)); */
+
+    checkArgument(StringUtils.isNotBlank(id));
+
+    this.apiVersion = apiVersion;
+    this.id = id;
+    this.data = data;
+    this.lensErrorTO = lensErrorTO;
+    this.httpStatusCode = httpStatusCode;
+  }
+
+  public boolean areValidStackTracesPresent() {
+    return (lensErrorTO == null) ? false : lensErrorTO.areValidStackTracesPresent();
+  }
+
+  public Status getHttpStatusCode() {
+    return this.httpStatusCode;
+  }
+
+  public boolean isSuccessResult() {
+    return lensErrorTO == null;
+  }
+
+  public boolean isErrorResult() {
+    return !isSuccessResult();
+  }
+
+  public int getErrorCode() {
+    checkState(isErrorResult());
+    return lensErrorTO.getCode();
+  }
+
+  public String getErrorMessage() {
+    checkState(isErrorResult());
+    return lensErrorTO.getMessage();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/result/LensErrorTO.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/result/LensErrorTO.java b/lens-api/src/main/java/org/apache/lens/api/result/LensErrorTO.java
new file mode 100644
index 0000000..a2aca36
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/result/LensErrorTO.java
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.api.result;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.Serializable;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+
+import org.apache.commons.lang.StringUtils;
+
+import lombok.*;
+
+/**
+ *
+ * Transport object for lens error information.
+ *
+ * @param <PAYLOAD>  represents type of error payload transferred in failure lens response
+ */
+@EqualsAndHashCode(exclude = { "stackTrace" })
+@ToString
+@NoArgsConstructor(access =  AccessLevel.PACKAGE)
+@XmlAccessorType(XmlAccessType.FIELD)
+public class LensErrorTO<PAYLOAD> implements Serializable {
+
+  @XmlElement
+  @Getter
+  private int code;
+
+  @XmlElement
+  @Getter
+  private String message;
+
+  @XmlElement
+  private String stackTrace;
+
+  @XmlElement
+  @Getter
+  private PAYLOAD payload;
+
+  @Getter
+  @XmlElementWrapper(name = "childErrors")
+  @XmlElement(name = "error")
+  private List<LensErrorTO> childErrors;
+
+  public static <PAYLOAD> LensErrorTO<PAYLOAD> composedOf(final int code, final String message,
+      final String stackTrace, final PAYLOAD payload, final List<LensErrorTO> childErrors) {
+
+    return new LensErrorTO<PAYLOAD>(code, message, stackTrace, payload, childErrors);
+  }
+
+  public static <PAYLOAD> LensErrorTO<PAYLOAD> composedOf(final int code, final String message,
+      final String stackTrace, final PAYLOAD payload) {
+
+    return new LensErrorTO<PAYLOAD>(code, message, stackTrace, payload, null);
+  }
+
+  public static LensErrorTO<NoErrorPayload> composedOf(final int code, final String message,
+      final String stackTrace) {
+
+    return new LensErrorTO<NoErrorPayload>(code, message, stackTrace, null, null);
+  }
+
+  public static LensErrorTO<NoErrorPayload> composedOf(final int code, final String message,
+      final String stackTrace, final List<LensErrorTO> childErrors) {
+
+    return new LensErrorTO<NoErrorPayload>(code, message, stackTrace, null, childErrors);
+  }
+
+  private LensErrorTO(final int code, final String message, final String stackTrace, final PAYLOAD errorPayload,
+      final List<LensErrorTO> childErrors) {
+
+    checkArgument(code > 0);
+    checkArgument(StringUtils.isNotBlank(message));
+    checkArgument(StringUtils.isNotBlank(stackTrace));
+
+    this.code = code;
+    this.message = message;
+    this.stackTrace = stackTrace;
+    this.payload = errorPayload;
+    this.childErrors = childErrors;
+  }
+
+  public boolean areValidStackTracesPresent() {
+
+    /* if stack trace of first level error is not valid, then return false */
+    if (StringUtils.isBlank(stackTrace)) {
+      return false;
+    }
+
+    /* validate stack traces of child Errors */
+    if (childErrors != null) {
+      for (LensErrorTO childError : childErrors) {
+        if (!childError.areValidStackTracesPresent()) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/result/LensJAXBContextResolver.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/result/LensJAXBContextResolver.java b/lens-api/src/main/java/org/apache/lens/api/result/LensJAXBContextResolver.java
new file mode 100644
index 0000000..5b48578
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/result/LensJAXBContextResolver.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.api.result;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import javax.ws.rs.ext.ContextResolver;
+import javax.ws.rs.ext.Provider;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+
+import org.apache.lens.api.error.ErrorCollection;
+import org.apache.lens.api.error.ErrorCollectionFactory;
+
+import com.google.common.collect.Sets;
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * @see javax.ws.rs.ext.ContextResolver
+ */
+@Provider
+@Slf4j
+public class LensJAXBContextResolver implements ContextResolver<JAXBContext> {
+
+  private Map<Class, JAXBContext> jaxbContextCache = new ConcurrentHashMap<Class, JAXBContext>();
+
+  @Override
+  public JAXBContext getContext(Class<?> type) {
+
+    JAXBContext jaxbContext = jaxbContextCache.get(type);
+
+    if (jaxbContext == null) {
+
+      log.debug("JAXB instance to be created for {}", type);
+      try {
+        if (type.equals(LensAPIResult.class)) {
+
+          ErrorCollection errorCollection = new ErrorCollectionFactory().createErrorCollection();
+          Set<Class> classesToBeBound = Sets.newHashSet(errorCollection.getErrorPayloadClasses());
+          log.debug("classesToBeBound:{}", classesToBeBound);
+          classesToBeBound.add(type);
+
+          Class[] classesToBeBoundArray = classesToBeBound.toArray(new Class[classesToBeBound.size()]);
+          jaxbContext = JAXBContext.newInstance(classesToBeBoundArray);
+        } else {
+
+          jaxbContext = JAXBContext.newInstance(type);
+        }
+        jaxbContextCache.put(type, jaxbContext);
+
+      } catch (JAXBException e) {
+        log.error("JAXBContext not initialized for "+type, e);
+      } catch (ClassNotFoundException e) {
+        log.error("JAXBContext not initialized for "+type, e);
+      }
+    }
+    return jaxbContext;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/result/NoErrorPayload.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/result/NoErrorPayload.java b/lens-api/src/main/java/org/apache/lens/api/result/NoErrorPayload.java
new file mode 100644
index 0000000..77810d3
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/result/NoErrorPayload.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.api.result;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import lombok.EqualsAndHashCode;
+
+/**
+ * NoErrorPayload type is to be used while creating LensResponse for success responses.
+ * Success Responses will not have any error payload.
+ *
+ */
+@XmlRootElement
+@EqualsAndHashCode
+public final class NoErrorPayload {
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/result/NoResultData.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/result/NoResultData.java b/lens-api/src/main/java/org/apache/lens/api/result/NoResultData.java
new file mode 100644
index 0000000..1b47d02
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/result/NoResultData.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.api.result;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * NoResultData type is to be used while creating LensAPIResult for error scenarios.
+ * Error scenarios will not have any data related to successful execution of API.
+ *
+ */
+@XmlRootElement
+public final class NoResultData {
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/java/org/apache/lens/api/result/PrettyPrintable.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/result/PrettyPrintable.java b/lens-api/src/main/java/org/apache/lens/api/result/PrettyPrintable.java
new file mode 100644
index 0000000..e6152b8
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/result/PrettyPrintable.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.api.result;
+
+/**
+ * Contents of a class which implements this interface can be represented in a pretty formatted string.
+ */
+public interface PrettyPrintable {
+
+  /**
+   * Returns the contents of this object in the form of a pretty formatted string.
+   *
+   * @return
+   */
+  String toPrettyString();
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-api/src/main/resources/lens-errors.conf
----------------------------------------------------------------------
diff --git a/lens-api/src/main/resources/lens-errors.conf b/lens-api/src/main/resources/lens-errors.conf
index 26bda1f..6130ad7 100644
--- a/lens-api/src/main/resources/lens-errors.conf
+++ b/lens-api/src/main/resources/lens-errors.conf
@@ -60,7 +60,7 @@ lensServerErrors = [
    errorCode = 2003
    httpStatusCode = ${BAD_REQUEST}
    errorMsg = "Provided Operation is not supported. Supported Operations are: [%s]"
-   payloadClass = org.apache.lens.server.query.SupportedQuerySubmitOperations
+   payloadClass = org.apache.lens.api.query.SupportedQuerySubmitOperations
  }
 
 ]

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-cli/src/main/java/org/apache/lens/cli/commands/LensQueryCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/main/java/org/apache/lens/cli/commands/LensQueryCommands.java b/lens-cli/src/main/java/org/apache/lens/cli/commands/LensQueryCommands.java
index 928531e..c48fabd 100644
--- a/lens-cli/src/main/java/org/apache/lens/cli/commands/LensQueryCommands.java
+++ b/lens-cli/src/main/java/org/apache/lens/cli/commands/LensQueryCommands.java
@@ -26,8 +26,14 @@ import java.util.UUID;
 import javax.ws.rs.core.Response;
 
 import org.apache.lens.api.query.*;
+import org.apache.lens.api.result.PrettyPrintable;
 import org.apache.lens.cli.commands.annotations.UserDocumentation;
 import org.apache.lens.client.LensClient;
+import org.apache.lens.client.exceptions.LensAPIException;
+import org.apache.lens.client.exceptions.LensBriefErrorException;
+import org.apache.lens.client.model.BriefError;
+import org.apache.lens.client.model.IdBriefErrorTemplate;
+import org.apache.lens.client.model.IdBriefErrorTemplateKey;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
@@ -74,17 +80,26 @@ public class LensQueryCommands extends BaseLensCommand {
     @CliOption(key = {"async"}, mandatory = false, unspecifiedDefaultValue = "false",
       specifiedDefaultValue = "true", help = "<async>") boolean async,
     @CliOption(key = {"name"}, mandatory = false, help = "<query-name>") String queryName) {
-    if (async) {
-      QueryHandle handle = getClient().executeQueryAsynch(sql, queryName);
-      return handle.getHandleId().toString();
-    } else {
-      try {
-        LensClient.LensClientResultSetWithStats result = getClient().getResults(sql, queryName);
-        return formatResultSet(result);
-      } catch (Throwable t) {
-        return t.getMessage();
+
+    PrettyPrintable cliOutput;
+
+    try {
+      if (async) {
+        QueryHandle queryHandle = getClient().executeQueryAsynch(sql, queryName).getData();
+        return queryHandle.getHandleIdString();
+      } else {
+        return formatResultSet(getClient().getResults(sql, queryName));
       }
+    } catch (final LensAPIException e) {
+
+      BriefError briefError = new BriefError(e.getLensAPIErrorCode(), e.getLensAPIErrorMessage());
+      cliOutput = new IdBriefErrorTemplate(IdBriefErrorTemplateKey.REQUEST_ID, e.getLensAPIRequestId(), briefError);
+
+    } catch (final LensBriefErrorException e) {
+      cliOutput = e.getIdBriefErrorTemplate();
     }
+
+    return cliOutput.toPrettyString();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-cli/src/test/java/org/apache/lens/cli/ExecuteQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/ExecuteQueryCommandIT.java b/lens-cli/src/test/java/org/apache/lens/cli/ExecuteQueryCommandIT.java
new file mode 100644
index 0000000..b3fcccb
--- /dev/null
+++ b/lens-cli/src/test/java/org/apache/lens/cli/ExecuteQueryCommandIT.java
@@ -0,0 +1,42 @@
+/**
+ * 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.cli;
+
+import static org.testng.Assert.assertTrue;
+
+import org.apache.lens.cli.commands.LensQueryCommands;
+import org.apache.lens.client.LensClient;
+
+import org.testng.annotations.Test;
+
+public class ExecuteQueryCommandIT extends LensCliApplicationTest {
+
+  @Test
+  public void testExecuteSyncQueryWithSyntaxError() {
+
+    LensQueryCommands lensQueryCommands = new LensQueryCommands();
+    lensQueryCommands.setClient(new LensClient());
+
+    final String actualResult = lensQueryCommands.executeQuery("mock-query", false, "testQuery");
+
+    assertTrue(actualResult.contains("Query Id: "));
+    assertTrue(actualResult.contains("\n" + "Error Code: 3001\n"
+        + "Error Message: Syntax Error: line 1:0 cannot recognize input near 'mock' '-' 'query'"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
index 97f0cf0..46e28a9 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensQueryCommands.java
@@ -225,7 +225,8 @@ public class TestLensQueryCommands extends LensCliApplicationTest {
     String sql = "cube select id,name from test_dim";
     long submitTime = System.currentTimeMillis();
     String qh = qCom.executeQuery(sql, true, "testQuery1");
-    String user = qCom.getClient().getLensStatement(new QueryHandle(UUID.fromString(qh))).getQuery().getSubmittedUser();
+    String user = qCom.getClient().getLensStatement(new QueryHandle(UUID.fromString(qh)))
+        .getQuery().getSubmittedUser();
     String result = qCom.getAllQueries("", "testQuery1", user, -1, Long.MAX_VALUE);
     // this is because previous query has run two query handle will be there
     assertTrue(result.contains(qh), result);

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/pom.xml
----------------------------------------------------------------------
diff --git a/lens-client/pom.xml b/lens-client/pom.xml
index e27a6ce..540c203 100644
--- a/lens-client/pom.xml
+++ b/lens-client/pom.xml
@@ -123,5 +123,25 @@
       <version>${project.version}</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+    </dependency>
+    <dependency>
+    <groupId>log4j</groupId>
+    <artifactId>log4j</artifactId>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/src/main/java/org/apache/lens/client/LensClient.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/LensClient.java b/lens-client/src/main/java/org/apache/lens/client/LensClient.java
index 7399d9e..51f586a 100644
--- a/lens-client/src/main/java/org/apache/lens/client/LensClient.java
+++ b/lens-client/src/main/java/org/apache/lens/client/LensClient.java
@@ -27,6 +27,12 @@ import javax.ws.rs.core.Response;
 import org.apache.lens.api.APIResult;
 import org.apache.lens.api.metastore.*;
 import org.apache.lens.api.query.*;
+import org.apache.lens.api.result.LensAPIResult;
+import org.apache.lens.client.exceptions.LensAPIException;
+import org.apache.lens.client.exceptions.LensBriefErrorException;
+import org.apache.lens.client.model.BriefError;
+import org.apache.lens.client.model.IdBriefErrorTemplate;
+import org.apache.lens.client.model.IdBriefErrorTemplateKey;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -86,13 +92,13 @@ public class LensClient {
     return mc;
   }
 
-  public QueryHandle executeQueryAsynch(String sql, String queryName) {
+  public LensAPIResult<QueryHandle> executeQueryAsynch(String sql, String queryName) throws LensAPIException {
     LOG.debug("Executing query " + sql);
-    statement.execute(sql, false, queryName);
+    LensAPIResult<QueryHandle> lensAPIResult = statement.execute(sql, false, queryName);
     LensQuery query = statement.getQuery();
     LOG.debug("Adding query to statementMap " + query.getQueryHandle());
     statementMap.put(query.getQueryHandle(), statement);
-    return query.getQueryHandle();
+    return lensAPIResult;
   }
 
   public Date getLatestDateOfCube(String cubeName, String timePartition) {
@@ -123,7 +129,7 @@ public class LensClient {
     }
   }
 
-  public LensClientResultSetWithStats getResults(String sql, String queryName) {
+  public LensClientResultSetWithStats getResults(String sql, String queryName) throws LensAPIException {
     LOG.debug("Executing query " + sql);
     statement.execute(sql, true, queryName);
     return getResultsFromStatement(statement);
@@ -132,8 +138,9 @@ public class LensClient {
   private LensClientResultSetWithStats getResultsFromStatement(LensStatement statement) {
     QueryStatus.Status status = statement.getStatus().getStatus();
     if (status != QueryStatus.Status.SUCCESSFUL) {
-      throw new IllegalStateException(statement.getStatus().getStatusMessage()
-        + " cause:" + statement.getStatus().getErrorMessage());
+      IdBriefErrorTemplate errorResult = new IdBriefErrorTemplate(IdBriefErrorTemplateKey.QUERY_ID,
+          statement.getQueryHandleString(), new BriefError(statement.getErrorCode(), statement.getErrorMessage()));
+      throw new LensBriefErrorException(errorResult);
     }
     LensClientResultSet result = null;
     if (statement.getStatus().isResultSetAvailable()) {

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/src/main/java/org/apache/lens/client/LensStatement.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/LensStatement.java b/lens-client/src/main/java/org/apache/lens/client/LensStatement.java
index 40e2b86..f7305fb 100644
--- a/lens-client/src/main/java/org/apache/lens/client/LensStatement.java
+++ b/lens-client/src/main/java/org/apache/lens/client/LensStatement.java
@@ -32,10 +32,8 @@ import org.apache.lens.api.APIResult;
 import org.apache.lens.api.query.*;
 import org.apache.lens.api.query.QueryStatus.Status;
 
-import org.apache.lens.api.response.LensJAXBContextResolver;
-import org.apache.lens.api.response.LensResponse;
-import org.apache.lens.api.response.NoErrorPayload;
-
+import org.apache.lens.api.result.LensAPIResult;
+import org.apache.lens.client.exceptions.LensAPIException;
 
 import org.apache.commons.lang.StringUtils;
 
@@ -71,9 +69,11 @@ public class LensStatement {
    * @param waitForQueryToComplete the wait for query to complete
    * @param queryName              the query name
    */
-  public void execute(String sql, boolean waitForQueryToComplete, String queryName) {
-    QueryHandle handle = executeQuery(sql, waitForQueryToComplete, queryName);
-    this.query = getQuery(handle);
+  public LensAPIResult<QueryHandle> execute(String sql, boolean waitForQueryToComplete,
+      String queryName) throws LensAPIException {
+    LensAPIResult<QueryHandle> lensAPIResult = executeQuery(sql, waitForQueryToComplete, queryName);
+    this.query = getQuery(lensAPIResult.getData());
+    return lensAPIResult;
   }
 
   /**
@@ -82,8 +82,8 @@ public class LensStatement {
    * @param sql       the sql
    * @param queryName the query name
    */
-  public void execute(String sql, String queryName) {
-    QueryHandle handle = executeQuery(sql, true, queryName);
+  public void execute(String sql, String queryName) throws LensAPIException {
+    QueryHandle handle = executeQuery(sql, true, queryName).getData();
     this.query = getQuery(handle);
   }
 
@@ -95,13 +95,15 @@ public class LensStatement {
    * @param queryName              the query name
    * @return the query handle
    */
-  public QueryHandle executeQuery(String sql, boolean waitForQueryToComplete, String queryName) {
-    QueryHandle handle = executeQuery(sql, queryName);
+  public LensAPIResult<QueryHandle> executeQuery(String sql, boolean waitForQueryToComplete,
+      String queryName) throws LensAPIException {
+
+    LensAPIResult<QueryHandle> lensAPIResult = executeQuery(sql, queryName);
 
     if (waitForQueryToComplete) {
-      waitForQueryToComplete(handle);
+      waitForQueryToComplete(lensAPIResult.getData());
     }
-    return handle;
+    return lensAPIResult;
   }
 
   /**
@@ -137,8 +139,8 @@ public class LensStatement {
     WebTarget target = getPreparedQueriesWebTarget(client);
 
     QueryPrepareHandle handle = target.request().post(
-      Entity.entity(prepareForm(sql, "PREPARE", queryName), MediaType.MULTIPART_FORM_DATA_TYPE),
-      QueryPrepareHandle.class);
+        Entity.entity(prepareForm(sql, "PREPARE", queryName), MediaType.MULTIPART_FORM_DATA_TYPE),
+        QueryPrepareHandle.class);
     getPreparedQuery(handle);
     return handle;
   }
@@ -288,14 +290,12 @@ public class LensStatement {
    * @param queryName the query name
    * @return the query handle
    */
-  private QueryHandle executeQuery(String sql, String queryName) {
+  private LensAPIResult<QueryHandle> executeQuery(String sql, String queryName) throws LensAPIException {
     if (!connection.isOpen()) {
       throw new IllegalStateException("Lens Connection has to be " + "established before querying");
     }
 
-    Client client = ClientBuilder.newBuilder().register(MultiPartFeature.class).register(LensJAXBContextResolver.class)
-      .build();
-
+    Client client = ClientBuilder.newBuilder().register(MultiPartFeature.class).build();
     FormDataMultiPart mp = new FormDataMultiPart();
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionid").build(), connection
       .getSessionHandle(), MediaType.APPLICATION_XML_TYPE));
@@ -306,8 +306,13 @@ public class LensStatement {
 
     WebTarget target = getQueryWebTarget(client);
 
-    return target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-      new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+    Response response = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE));
+
+    if (response.getStatus() == Response.Status.OK.getStatusCode()) {
+      return response.readEntity(new GenericType<LensAPIResult<QueryHandle>>() {});
+    }
+
+    throw new LensAPIException(response.readEntity(LensAPIResult.class));
   }
 
   /**
@@ -358,7 +363,7 @@ public class LensStatement {
     WebTarget target = getQueryWebTarget(client);
 
     QueryPlan handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-      new GenericType<LensResponse<QueryPlan, NoErrorPayload>>() {}).getData();
+        new GenericType<LensAPIResult<QueryPlan>>() {}).getData();
     return handle;
   }
 
@@ -396,8 +401,7 @@ public class LensStatement {
     WebTarget target = getPreparedQueriesWebTarget(client);
     List<QueryPrepareHandle> handles = target.queryParam("sessionid", connection.getSessionHandle())
       .queryParam("user", userName).queryParam("queryName", queryName).queryParam("fromDate", fromDate)
-      .queryParam("toDate", toDate).request().get(new GenericType<List<QueryPrepareHandle>>() {
-      });
+      .queryParam("toDate", toDate).request().get(new GenericType<List<QueryPrepareHandle>>() {});
     return handles;
   }
 
@@ -561,4 +565,16 @@ public class LensStatement {
   public LensQuery getQuery() {
     return this.query;
   }
+
+  public int getErrorCode() {
+    return this.query.getErrorCode();
+  }
+
+  public String getErrorMessage() {
+    return this.query.getErrorMessage();
+  }
+
+  public String getQueryHandleString() {
+    return this.query.getQueryHandleString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/src/main/java/org/apache/lens/client/exceptions/LensAPIException.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/exceptions/LensAPIException.java b/lens-client/src/main/java/org/apache/lens/client/exceptions/LensAPIException.java
new file mode 100644
index 0000000..fcc53c5
--- /dev/null
+++ b/lens-client/src/main/java/org/apache/lens/client/exceptions/LensAPIException.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.client.exceptions;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.lens.api.result.LensAPIResult;
+
+import lombok.ToString;
+
+@ToString
+public class LensAPIException extends Exception {
+
+  private LensAPIResult errorResult;
+
+  public LensAPIException(final LensAPIResult lensAPIErrorResult) {
+    checkState(lensAPIErrorResult.isErrorResult());
+    this.errorResult = lensAPIErrorResult;
+  }
+
+  public int getLensAPIErrorCode() {
+    return this.errorResult.getErrorCode();
+  }
+
+  public String getLensAPIErrorMessage() {
+    return this.errorResult.getErrorMessage();
+  }
+
+  public String getLensAPIRequestId() {
+    return this.errorResult.getId();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/src/main/java/org/apache/lens/client/exceptions/LensBriefErrorException.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/exceptions/LensBriefErrorException.java b/lens-client/src/main/java/org/apache/lens/client/exceptions/LensBriefErrorException.java
new file mode 100644
index 0000000..ddfd777
--- /dev/null
+++ b/lens-client/src/main/java/org/apache/lens/client/exceptions/LensBriefErrorException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.client.exceptions;
+
+import org.apache.lens.client.model.IdBriefErrorTemplate;
+
+import lombok.Getter;
+import lombok.NonNull;
+
+public class LensBriefErrorException extends RuntimeException {
+
+  @Getter
+  private final IdBriefErrorTemplate idBriefErrorTemplate;
+
+  public LensBriefErrorException(@NonNull final IdBriefErrorTemplate idBriefErrorTemplate) {
+    this.idBriefErrorTemplate = idBriefErrorTemplate;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/src/main/java/org/apache/lens/client/exceptions/LensClientException.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/exceptions/LensClientException.java b/lens-client/src/main/java/org/apache/lens/client/exceptions/LensClientException.java
index 1150726..1c03be7 100644
--- a/lens-client/src/main/java/org/apache/lens/client/exceptions/LensClientException.java
+++ b/lens-client/src/main/java/org/apache/lens/client/exceptions/LensClientException.java
@@ -18,6 +18,8 @@
  */
 package org.apache.lens.client.exceptions;
 
+import org.apache.lens.api.result.LensAPIResult;
+
 /**
  * The Class LensClientException.
  */
@@ -26,12 +28,7 @@ public class LensClientException extends RuntimeException {
   /** The Constant serialVersionUID. */
   private static final long serialVersionUID = 1L;
 
-  /** The message. */
-  private final String message;
-
-  /** The cause. */
-  private Exception cause;
-
+  private LensAPIResult lensAPIErrorResult;
   /**
    * Instantiates a new lens client exception.
    *
@@ -39,8 +36,7 @@ public class LensClientException extends RuntimeException {
    * @param cause   the cause
    */
   public LensClientException(String message, Exception cause) {
-    this.message = message;
-    this.cause = cause;
+    super(message, cause);
   }
 
   /**
@@ -49,16 +45,6 @@ public class LensClientException extends RuntimeException {
    * @param message the message
    */
   public LensClientException(String message) {
-    this.message = message;
-  }
-
-  @Override
-  public String getMessage() {
-    return message;
-  }
-
-  @Override
-  public Exception getCause() {
-    return cause;
+    super(message);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/src/main/java/org/apache/lens/client/jdbc/LensJdbcStatement.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/jdbc/LensJdbcStatement.java b/lens-client/src/main/java/org/apache/lens/client/jdbc/LensJdbcStatement.java
index 95d0ff7..10f7155 100644
--- a/lens-client/src/main/java/org/apache/lens/client/jdbc/LensJdbcStatement.java
+++ b/lens-client/src/main/java/org/apache/lens/client/jdbc/LensJdbcStatement.java
@@ -21,10 +21,14 @@ package org.apache.lens.client.jdbc;
 import java.sql.*;
 
 import org.apache.lens.client.LensStatement;
+import org.apache.lens.client.exceptions.LensAPIException;
+
+import lombok.extern.slf4j.Slf4j;
 
 /**
  * The Class LensJdbcStatement.
  */
+@Slf4j
 public class LensJdbcStatement implements Statement {
 
   /** The connection. */
@@ -53,7 +57,11 @@ public class LensJdbcStatement implements Statement {
    */
   @Override
   public ResultSet executeQuery(String s) throws SQLException {
-    statement.execute(s, null);
+    try {
+      statement.execute(s, null);
+    } catch (LensAPIException e) {
+      log.error("Execution Failed for Statement:{}", s, e);
+    }
     return new LensJdbcResultSet(statement.getResultSet(), statement.getResultSetMetaData(), this);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/src/main/java/org/apache/lens/client/model/BriefError.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/model/BriefError.java b/lens-client/src/main/java/org/apache/lens/client/model/BriefError.java
new file mode 100644
index 0000000..f2b37ea
--- /dev/null
+++ b/lens-client/src/main/java/org/apache/lens/client/model/BriefError.java
@@ -0,0 +1,47 @@
+/**
+ * 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.client.model;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.lens.api.result.PrettyPrintable;
+
+import org.apache.commons.lang.StringUtils;
+
+public class BriefError implements PrettyPrintable {
+
+  private final int errorCode;
+  private final String errorMsg;
+
+  public BriefError(final int errorCode, final String errorMsg) {
+
+    checkArgument(errorCode > 0);
+    checkArgument(StringUtils.isNotBlank(errorMsg));
+    this.errorCode = errorCode;
+    this.errorMsg = errorMsg;
+  }
+
+  @Override
+  public String toPrettyString() {
+
+    StringBuilder sb = new StringBuilder("Error Code: ").append(this.errorCode).append("\n").append("Error Message: ")
+        .append(this.errorMsg);
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/src/main/java/org/apache/lens/client/model/IdBriefErrorTemplate.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/model/IdBriefErrorTemplate.java b/lens-client/src/main/java/org/apache/lens/client/model/IdBriefErrorTemplate.java
new file mode 100644
index 0000000..8a485b5
--- /dev/null
+++ b/lens-client/src/main/java/org/apache/lens/client/model/IdBriefErrorTemplate.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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.client.model;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.lens.api.result.PrettyPrintable;
+
+import org.apache.commons.lang.StringUtils;
+
+import lombok.NonNull;
+
+public class IdBriefErrorTemplate implements PrettyPrintable {
+
+  private final IdBriefErrorTemplateKey idKey;
+  private final String idValue;
+  private final BriefError briefError;
+
+  public IdBriefErrorTemplate(@NonNull final IdBriefErrorTemplateKey idKey, final String idValue,
+      @NonNull BriefError briefError) {
+
+    checkArgument(StringUtils.isNotBlank(idValue));
+    this.idKey = idKey;
+    this.idValue = idValue;
+    this.briefError = briefError;
+  }
+
+  @Override
+  public String toPrettyString() {
+
+    StringBuilder sb = new StringBuilder(idKey.getConstant()).append(": ").append(this.idValue).append("\n")
+        .append(this.briefError.toPrettyString());
+
+    return sb.toString();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/src/main/java/org/apache/lens/client/model/IdBriefErrorTemplateKey.java
----------------------------------------------------------------------
diff --git a/lens-client/src/main/java/org/apache/lens/client/model/IdBriefErrorTemplateKey.java b/lens-client/src/main/java/org/apache/lens/client/model/IdBriefErrorTemplateKey.java
new file mode 100644
index 0000000..41e7177
--- /dev/null
+++ b/lens-client/src/main/java/org/apache/lens/client/model/IdBriefErrorTemplateKey.java
@@ -0,0 +1,35 @@
+/**
+ * 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.client.model;
+
+public enum IdBriefErrorTemplateKey {
+
+  REQUEST_ID("Request Id"),
+  QUERY_ID("Query Id");
+
+  IdBriefErrorTemplateKey(final String key) {
+    this.key = key;
+  }
+
+  public String getConstant() {
+    return this.key;
+  }
+
+  private final String key;
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/src/test/java/org/apache/lens/client/model/BriefErrorTest.java
----------------------------------------------------------------------
diff --git a/lens-client/src/test/java/org/apache/lens/client/model/BriefErrorTest.java b/lens-client/src/test/java/org/apache/lens/client/model/BriefErrorTest.java
new file mode 100644
index 0000000..90e2fbe
--- /dev/null
+++ b/lens-client/src/test/java/org/apache/lens/client/model/BriefErrorTest.java
@@ -0,0 +1,60 @@
+/**
+ * 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.client.model;
+
+import static org.testng.Assert.assertEquals;
+
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+public class BriefErrorTest {
+
+  @Test
+  public void testToPrettyString() {
+
+    final int testErrCode = 1001;
+    final String testErrorMsg = "Test Error Msg, adfg-asdfk $ %, \n";
+    BriefError briefError = new BriefError(testErrCode, testErrorMsg);
+
+    final String actualPrettyString = briefError.toPrettyString();
+    final String expectedPrettyString = "Error Code: 1001\nError Message: Test Error Msg, adfg-asdfk $ %, \n";
+
+    assertEquals(actualPrettyString, expectedPrettyString);
+  }
+
+  @DataProvider(name="dpInvalidErrorCodes")
+  public Object[][] dpInvalidErrorCodes() {
+    return new Object[][] {{-1}, {0}};
+  }
+
+  @Test(dataProvider = "dpInvalidStrings", expectedExceptions = IllegalArgumentException.class)
+  public void testBriefErrorMustNotAcceptInvalidErrorCodes(final int invalidErrCode) {
+    new BriefError(invalidErrCode, "Valid Error Message");
+  }
+
+  @DataProvider(name="dpInvalidStrings")
+  public Object[][] dpInvalidStrings() {
+    return new Object[][] {{null}, {""}, {"  "}};
+  }
+
+  @Test(dataProvider = "dpInvalidStrings", expectedExceptions = IllegalArgumentException.class)
+  public void testBriefErrorMustNotAcceptInvalidErrorMsg(final String invalidErrMsg) {
+    new BriefError(1001, invalidErrMsg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-client/src/test/java/org/apache/lens/client/model/IdBriefErrorTemplateTest.java
----------------------------------------------------------------------
diff --git a/lens-client/src/test/java/org/apache/lens/client/model/IdBriefErrorTemplateTest.java b/lens-client/src/test/java/org/apache/lens/client/model/IdBriefErrorTemplateTest.java
new file mode 100644
index 0000000..9d9ec0e
--- /dev/null
+++ b/lens-client/src/test/java/org/apache/lens/client/model/IdBriefErrorTemplateTest.java
@@ -0,0 +1,74 @@
+/**
+ * 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.client.model;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+public class IdBriefErrorTemplateTest {
+
+  @Test
+  public void testToPrettyString() {
+
+    /* Setup Dependencies */
+    final IdBriefErrorTemplateKey testQueryIdKey = IdBriefErrorTemplateKey.QUERY_ID;
+    final String testQueryIdValue = "TestQueryIdValue";
+    final BriefError mockBriefError = mock(BriefError.class);
+
+    /* Stubbing */
+    final String testBriefErrorPrettyString = "TestLensBriefErrorPrettyString";
+    when(mockBriefError.toPrettyString()).thenReturn(testBriefErrorPrettyString);
+
+    /* Execution */
+    IdBriefErrorTemplate template = new IdBriefErrorTemplate(testQueryIdKey, testQueryIdValue, mockBriefError);
+    final String actualPrettyString = template.toPrettyString();
+
+    /* Verfication */
+    final String expectedPrettyString = "Query Id: TestQueryIdValue\nTestLensBriefErrorPrettyString";
+    assertEquals(actualPrettyString, expectedPrettyString);
+  }
+
+  @DataProvider(name="dpInvalidStrings")
+  public Object[][] dpInvalidStrings() {
+    return new Object[][] {{null}, {""}, {"  "}};
+  }
+
+  @Test(expectedExceptions = NullPointerException.class)
+  public void testIdBriefErrorTemplateMustNotAcceptNullIdKey() {
+
+    final BriefError mockBriefError = mock(BriefError.class);
+    new IdBriefErrorTemplate(null, "ValidIdValue", mockBriefError);
+  }
+
+  @Test(dataProvider = "dpInvalidStrings", expectedExceptions = IllegalArgumentException.class)
+  public void testIdBriefErrorTemplateMustNotAcceptInvalidIdValue(final String invalidIdValue) {
+
+    final BriefError mockBriefError = mock(BriefError.class);
+    new IdBriefErrorTemplate(IdBriefErrorTemplateKey.QUERY_ID, invalidIdValue, mockBriefError);
+  }
+
+  @Test(expectedExceptions = NullPointerException.class)
+  public void testIdBriefErrorTemplateMustNotAcceptNullBriefError() {
+    new IdBriefErrorTemplate(IdBriefErrorTemplateKey.QUERY_ID, "ValidIdValue", null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-cube/src/main/java/org/apache/lens/cube/error/ColUnAvailableInTimeRangeException.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/error/ColUnAvailableInTimeRangeException.java b/lens-cube/src/main/java/org/apache/lens/cube/error/ColUnAvailableInTimeRangeException.java
index 99a78f6..dd3bb72 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/error/ColUnAvailableInTimeRangeException.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/error/ColUnAvailableInTimeRangeException.java
@@ -22,7 +22,7 @@ import static org.apache.lens.cube.error.LensCubeErrorCode.COLUMN_UNAVAILABLE_IN
 
 import org.apache.lens.api.error.ErrorCollection;
 import org.apache.lens.api.error.LensError;
-import org.apache.lens.api.response.LensErrorTO;
+import org.apache.lens.api.result.LensErrorTO;
 import org.apache.lens.server.api.error.LensException;
 
 import lombok.EqualsAndHashCode;

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-cube/src/main/java/org/apache/lens/cube/error/FieldsCannotBeQueriedTogetherException.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/error/FieldsCannotBeQueriedTogetherException.java b/lens-cube/src/main/java/org/apache/lens/cube/error/FieldsCannotBeQueriedTogetherException.java
index f246246..65b96d7 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/error/FieldsCannotBeQueriedTogetherException.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/error/FieldsCannotBeQueriedTogetherException.java
@@ -22,7 +22,7 @@ import static org.apache.lens.cube.error.LensCubeErrorCode.FIELDS_CANNOT_BE_QUER
 
 import org.apache.lens.api.error.ErrorCollection;
 import org.apache.lens.api.error.LensError;
-import org.apache.lens.api.response.LensErrorTO;
+import org.apache.lens.api.result.LensErrorTO;
 import org.apache.lens.server.api.error.LensException;
 
 import lombok.EqualsAndHashCode;

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-examples/src/main/java/org/apache/lens/examples/SampleQueries.java
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/java/org/apache/lens/examples/SampleQueries.java b/lens-examples/src/main/java/org/apache/lens/examples/SampleQueries.java
index 6c820a4..6b662e7 100644
--- a/lens-examples/src/main/java/org/apache/lens/examples/SampleQueries.java
+++ b/lens-examples/src/main/java/org/apache/lens/examples/SampleQueries.java
@@ -29,6 +29,7 @@ import org.apache.lens.api.query.*;
 import org.apache.lens.client.LensClientSingletonWrapper;
 import org.apache.lens.client.LensMetadataClient;
 import org.apache.lens.client.LensStatement;
+import org.apache.lens.client.exceptions.LensAPIException;
 
 import org.apache.commons.lang.StringUtils;
 
@@ -99,7 +100,7 @@ public class SampleQueries {
    *
    * @throws IOException Signals that an I/O exception has occurred.
    */
-  public void queryAll() throws IOException {
+  public void queryAll() throws IOException, LensAPIException {
     runQueries("dimension-queries.sql");
     runQueries("cube-queries.sql");
     System.out.println("Successful queries " + success + " out of " + total + "queries");
@@ -117,7 +118,7 @@ public class SampleQueries {
    * @param fileName the file name
    * @throws IOException Signals that an I/O exception has occurred.
    */
-  public void runQueries(String fileName) throws IOException {
+  public void runQueries(String fileName) throws IOException, LensAPIException {
     InputStream file = SampleMetastore.class.getClassLoader().getResourceAsStream(fileName);
     BufferedReader reader = new BufferedReader(new InputStreamReader(file, "UTF-8"));
     String query;
@@ -131,7 +132,7 @@ public class SampleQueries {
       }
       total++;
       System.out.println("Query:" + query);
-      QueryHandle handle = queryClient.executeQuery(query, true, null);
+      QueryHandle handle = queryClient.executeQuery(query, true, null).getData();
       System.out.println("Status:" + queryClient.getQuery().getStatus());
       System.out.println("Total time in millis:"
         + (queryClient.getQuery().getFinishTime() - queryClient.getQuery().getSubmissionTime()));

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-ml-lib/src/main/java/org/apache/lens/ml/impl/LensMLImpl.java
----------------------------------------------------------------------
diff --git a/lens-ml-lib/src/main/java/org/apache/lens/ml/impl/LensMLImpl.java b/lens-ml-lib/src/main/java/org/apache/lens/ml/impl/LensMLImpl.java
index de76603..50c22f4 100644
--- a/lens-ml-lib/src/main/java/org/apache/lens/ml/impl/LensMLImpl.java
+++ b/lens-ml-lib/src/main/java/org/apache/lens/ml/impl/LensMLImpl.java
@@ -38,8 +38,7 @@ 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.response.LensResponse;
-import org.apache.lens.api.response.NoErrorPayload;
+import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.ml.algo.api.MLAlgo;
 import org.apache.lens.ml.algo.api.MLDriver;
 import org.apache.lens.ml.algo.api.MLModel;
@@ -647,7 +646,7 @@ public class LensMLImpl implements LensML {
         MediaType.APPLICATION_XML_TYPE));
 
       final QueryHandle handle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-          new GenericType<LensResponse<QueryHandle, NoErrorPayload>>() {}).getData();
+          new GenericType<LensAPIResult<QueryHandle>>() {}).getData();
 
       LensQuery ctx = target.path(handle.toString()).queryParam("sessionid", sessionHandle).request()
         .get(LensQuery.class);

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/d1d99708/lens-ml-lib/src/main/java/org/apache/lens/rdd/LensRDDClient.java
----------------------------------------------------------------------
diff --git a/lens-ml-lib/src/main/java/org/apache/lens/rdd/LensRDDClient.java b/lens-ml-lib/src/main/java/org/apache/lens/rdd/LensRDDClient.java
index fe4d926..0efa10d 100644
--- a/lens-ml-lib/src/main/java/org/apache/lens/rdd/LensRDDClient.java
+++ b/lens-ml-lib/src/main/java/org/apache/lens/rdd/LensRDDClient.java
@@ -28,6 +28,7 @@ import java.util.UUID;
 import org.apache.lens.api.query.*;
 import org.apache.lens.client.LensClient;
 import org.apache.lens.client.LensClientResultSet;
+import org.apache.lens.client.exceptions.LensAPIException;
 import org.apache.lens.ml.algo.spark.HiveTableRDD;
 import org.apache.lens.server.api.error.LensException;
 
@@ -169,8 +170,8 @@ public class LensRDDClient {
    * @return the query handle
    * @throws LensException the lens exception
    */
-  public QueryHandle createLensRDDAsync(String query) throws LensException {
-    return getClient().executeQueryAsynch(query, "");
+  public QueryHandle createLensRDDAsync(String query) throws LensAPIException {
+    return getClient().executeQueryAsynch(query, "").getData();
   }
 
   /**
@@ -305,7 +306,7 @@ public class LensRDDClient {
    * @return the lens rdd result
    * @throws LensException the lens exception
    */
-  public LensRDDResult createLensRDD(String query) throws LensException {
+  public LensRDDResult createLensRDD(String query) throws LensAPIException, LensException {
     QueryHandle queryHandle = createLensRDDAsync(query);
     while (!isReadyForRDD(queryHandle)) {
       try {