You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by ra...@apache.org on 2015/10/09 06:17:17 UTC

[12/50] [abbrv] lens git commit: Revert "LENS-737 : Throw single error out with LensMultiException"

http://git-wip-us.apache.org/repos/asf/lens/blob/fb235faa/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 ac1c558..603d7cb 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
@@ -31,9 +31,7 @@ import org.apache.lens.api.error.ErrorCollection;
 import org.apache.lens.api.error.LensError;
 import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.api.result.LensErrorTO;
-import org.apache.lens.server.api.LensErrorInfo;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.exception.ExceptionUtils;
 
 import lombok.Getter;
@@ -43,27 +41,13 @@ import lombok.NonNull;
  * The Class LensException.
  */
 @SuppressWarnings("serial")
-public class LensException extends Exception implements Comparable<LensException> {
+public class LensException extends Exception {
 
   private static final int DEFAULT_LENS_EXCEPTION_ERROR_CODE = INTERNAL_SERVER_ERROR.getValue();
-  private static final int DEFAULT_LENS_EXCEPTION_WEIGHT = 0;
-
-  private static  LensErrorInfo defaultErrorInfo =
-      new LensErrorInfo(DEFAULT_LENS_EXCEPTION_ERROR_CODE, DEFAULT_LENS_EXCEPTION_WEIGHT, INTERNAL_SERVER_ERROR.name());
-
-  private Object[] errorMsgFormattingArgs = new Object[0];
 
   @Getter
-  private final LensErrorInfo errorInfo;
-
-  public int getErrorCode() {
-    return errorInfo.getErrorCode();
-  }
-
-  public int getErrorWeight() {
-    return errorInfo.getErrorWeight();
-  }
-
+  private final int errorCode;
+  private Object[] errorMsgFormattingArgs = new Object[0];
 
   /**
    * The lensResponse prepared by {@link #buildLensErrorResponse(ErrorCollection, String, String)}
@@ -77,7 +61,7 @@ public class LensException extends Exception implements Comparable<LensException
    * @see Exception#Exception(String)
    */
   public LensException(String errorMsg) {
-    this(errorMsg, defaultErrorInfo);
+    this(errorMsg, DEFAULT_LENS_EXCEPTION_ERROR_CODE);
   }
 
   /**
@@ -86,7 +70,7 @@ public class LensException extends Exception implements Comparable<LensException
    * @see Exception#Exception(String, Throwable)
    */
   public LensException(String errorMsg, Throwable cause) {
-    this(errorMsg, defaultErrorInfo, cause);
+    this(errorMsg, DEFAULT_LENS_EXCEPTION_ERROR_CODE, cause);
   }
 
   /**
@@ -95,7 +79,7 @@ public class LensException extends Exception implements Comparable<LensException
    * @see Exception#Exception()
    */
   public LensException() {
-    this(null, defaultErrorInfo);
+    this(null, DEFAULT_LENS_EXCEPTION_ERROR_CODE);
   }
 
   /**
@@ -104,100 +88,79 @@ public class LensException extends Exception implements Comparable<LensException
    * @see Exception#Exception(Throwable)
    */
   public LensException(Throwable cause) {
-    this(defaultErrorInfo, cause);
+    this(null, DEFAULT_LENS_EXCEPTION_ERROR_CODE, cause);
   }
 
   /**
-   * Constructs a new Lens Exception with error info.
+   * Constructs a new Lens Exception with error code.
    *
    * @see Exception#Exception()
    */
-  public LensException(final LensErrorInfo errorInfo) {
-    this(null, errorInfo);
+  public LensException(final int errorCode) {
+    this(null, errorCode);
   }
 
   /**
-   * Constructs a new Lens Exception with error msg and error info.
+   * Constructs a new Lens Exception with error msg and error code.
    *
    * @see Exception#Exception()
    */
-  public LensException(final String errorMsg, final LensErrorInfo errorInfo) {
-    this(errorMsg, errorInfo, null);
+  public LensException(final String errorMsg, final int errorCode) {
+    this(errorMsg, errorCode, null);
   }
 
   /**
-   * Constructs a new Lens Exception with error info, cause and error msg formatting arguments.
+   * Constructs a new Lens Exception with error code, cause and error msg formatting arguments.
    *
    * @see Exception#Exception(Throwable)
    */
-  public LensException(final LensErrorInfo errorInfo, final Throwable cause,
-      @NonNull final Object... errorMsgFormattingArgs) {
-    this(null, errorInfo, cause, errorMsgFormattingArgs);
+  public LensException(final int errorCode, final Throwable cause, @NonNull final Object... errorMsgFormattingArgs) {
+    this(null, errorCode, cause, errorMsgFormattingArgs);
   }
 
   /**
-   * Constructs a new Lens Exception with error info and error msg formatting arguments.
+   * Constructs a new Lens Exception with error code and error msg formatting arguments.
    *
    * @see Exception#Exception(Throwable)
    */
-  public LensException(final LensErrorInfo errorInfo, @NonNull final Object... errorMsgFormattingArgs) {
-    this(null, errorInfo, null, errorMsgFormattingArgs);
+  public LensException(final int errorCode, @NonNull final Object... errorMsgFormattingArgs) {
+    this(null, errorCode, null, errorMsgFormattingArgs);
   }
 
+
   /**
-   * Constructs a new Lens Exception with exception error message, error info, cause and error msg formatting arguments.
+   * Constructs a new Lens Exception with exception error message, error code, cause and error msg formatting arguments.
    *
    * @see Exception#Exception(Throwable)
    */
-  public LensException(final String errorMsg, final LensErrorInfo errorInfo, final Throwable cause,
+  public LensException(final String errorMsg, final int errorcode, final Throwable cause,
     @NonNull final Object... errorMsgFormattingArgs) {
 
-    super(getErrorMessage(errorMsg, errorInfo, errorMsgFormattingArgs), cause);
-    checkArgument(errorInfo.getErrorCode() > 0);
+    super(errorMsg, cause);
+    checkArgument(errorcode > 0);
 
-    this.errorInfo =  errorInfo;
+    this.errorCode = errorcode;
     this.errorMsgFormattingArgs = errorMsgFormattingArgs;
   }
 
-  private static String getErrorMessage(final String errorMsg, final LensErrorInfo errorInfo,
-      @NonNull final Object... errorMsgFormattingArgs) {
-
-    if (StringUtils.isBlank(errorMsg)) {
-      StringBuilder error = new StringBuilder(errorInfo.getErrorName());
-      if (errorMsgFormattingArgs != null && errorMsgFormattingArgs.length != 0) {
-        error.append(Arrays.asList(errorMsgFormattingArgs));
-      }
-      return error.toString();
-    }
-    return errorMsg;
-  }
-
-  /**
-   * Copy Constructor
-   * @param e
-   */
-  public LensException(LensException e) {
-    this(e.getMessage(), e.getErrorInfo(), e.getCause(), e.errorMsgFormattingArgs);
-  }
-
   public final void buildLensErrorResponse(final ErrorCollection errorCollection,
     final String apiVersion, final String id) {
 
-    final LensError lensError = errorCollection.getLensError(getErrorCode());
+    final LensError lensError = errorCollection.getLensError(errorCode);
     final LensErrorTO lensErrorTO = buildLensErrorTO(errorCollection, lensError);
     lensAPIResult = LensAPIResult.composedOf(apiVersion, id, lensErrorTO, lensError.getHttpStatusCode());
   }
 
   public final LensErrorTO buildLensErrorTO(final ErrorCollection errorCollection) {
 
-    final LensError lensError = errorCollection.getLensError(getErrorCode());
+    final LensError lensError = errorCollection.getLensError(errorCode);
     return buildLensErrorTO(errorCollection, lensError);
   }
 
   protected LensErrorTO buildLensErrorTO(final ErrorCollection errorCollection, final String errorMsg,
     final String stackTrace) {
 
-    return LensErrorTO.composedOf(getErrorCode(), errorMsg, stackTrace);
+    return LensErrorTO.composedOf(errorCode, errorMsg, stackTrace);
   }
 
   private LensErrorTO buildLensErrorTO(final ErrorCollection errorCollection, final LensError lensError) {
@@ -219,7 +182,7 @@ public class LensException extends Exception implements Comparable<LensException
     }
 
     LensException e = (LensException) o;
-    if (errorInfo.equals(e.errorInfo) && isErrorMsgEqual(e)
+    if (errorCode == e.errorCode && isErrorMsgEqual(e)
       && Arrays.deepEquals(errorMsgFormattingArgs, e.errorMsgFormattingArgs)) {
       return true;
     }
@@ -253,7 +216,7 @@ public class LensException extends Exception implements Comparable<LensException
     final int PRIME = 59;
     int result = 1;
 
-    result = result * PRIME + errorInfo.hashCode();
+    result = result * PRIME + errorCode;
     result = result * PRIME + (this.getMessage() == null ? 0 : this.getMessage().hashCode());
     result = result * PRIME + Arrays.deepHashCode(errorMsgFormattingArgs);
     return result;
@@ -265,9 +228,4 @@ public class LensException extends Exception implements Comparable<LensException
     }
     return new LensException(e);
   }
-
-  @Override
-  public int compareTo(LensException e) {
-    return this.getErrorWeight() - e.getErrorWeight();
-  }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/fb235faa/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 5248489..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
@@ -18,7 +18,10 @@
  */
 package org.apache.lens.server.api.error;
 
-import java.util.Collections;
+import static org.apache.lens.api.error.LensCommonErrorCode.INTERNAL_SERVER_ERROR;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
 import java.util.LinkedList;
 import java.util.List;
 
@@ -26,7 +29,6 @@ import org.apache.lens.api.error.ErrorCollection;
 import org.apache.lens.api.result.LensErrorTO;
 
 import com.google.common.collect.ImmutableList;
-
 import lombok.AccessLevel;
 import lombok.Getter;
 import lombok.NonNull;
@@ -48,14 +50,14 @@ public class LensMultiCauseException extends LensException {
   @Getter(AccessLevel.PROTECTED)
   private final ImmutableList<LensException> causes;
 
-  public LensMultiCauseException(@NonNull final ImmutableList<LensException> excpList) {
-    super(getAppropriateError(excpList));
-    this.causes = excpList;
-  }
+  public LensMultiCauseException(final String errMsg, @NonNull
+    final ImmutableList<LensException> causes) {
+
+    super(errMsg, INTERNAL_SERVER_ERROR.getValue());
+    checkArgument(causes.size() >= 2, "LensMultiCauseException should only be created when there are atleast "
+        + "two causes. An instance of LensException should be sufficient if there is only one cause.");
 
-  // Get appropriate error code
-  public static LensException getAppropriateError(final ImmutableList<LensException> excpList) {
-    return Collections.max(excpList);
+    this.causes = causes;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lens/blob/fb235faa/lens-server/src/main/java/org/apache/lens/server/error/LensServerErrorCode.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/error/LensServerErrorCode.java b/lens-server/src/main/java/org/apache/lens/server/error/LensServerErrorCode.java
index dc20f0f..917eba3 100644
--- a/lens-server/src/main/java/org/apache/lens/server/error/LensServerErrorCode.java
+++ b/lens-server/src/main/java/org/apache/lens/server/error/LensServerErrorCode.java
@@ -18,22 +18,20 @@
  */
 package org.apache.lens.server.error;
 
-import org.apache.lens.server.api.LensErrorInfo;
-
 public enum LensServerErrorCode {
 
-  SESSION_ID_NOT_PROVIDED(2001, 0),
-  NULL_OR_EMPTY_OR_BLANK_QUERY(2002, 0),
-  UNSUPPORTED_QUERY_SUBMIT_OPERATION(2003, 0);
+  SESSION_ID_NOT_PROVIDED(2001),
+  NULL_OR_EMPTY_OR_BLANK_QUERY(2002),
+  UNSUPPORTED_QUERY_SUBMIT_OPERATION(2003);
 
-  public LensErrorInfo getLensErrorInfo() {
-    return this.errorInfo;
+  public int getValue() {
+    return this.errorCode;
   }
 
-  LensServerErrorCode(final int code, final int weight) {
-    this.errorInfo = new LensErrorInfo(code, weight, name());
+  private LensServerErrorCode(final int code) {
+    this.errorCode = code;
   }
 
-  private final LensErrorInfo errorInfo;
+  private final int errorCode;
 
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/fb235faa/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 d90f8fd..366b306 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
@@ -31,11 +31,11 @@ public class UnSupportedQuerySubmitOpException extends LensException {
   private final SupportedQuerySubmitOperations supportedOps = new SupportedQuerySubmitOperations();
 
   public UnSupportedQuerySubmitOpException() {
-    super(UNSUPPORTED_QUERY_SUBMIT_OPERATION.getLensErrorInfo());
+    super(UNSUPPORTED_QUERY_SUBMIT_OPERATION.getValue());
   }
 
   public UnSupportedQuerySubmitOpException(final Throwable cause) {
-    super(UNSUPPORTED_QUERY_SUBMIT_OPERATION.getLensErrorInfo(), cause);
+    super(UNSUPPORTED_QUERY_SUBMIT_OPERATION.getValue(), cause);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lens/blob/fb235faa/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 23cc748..c29a1ac 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
@@ -1275,7 +1275,7 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
           final LensException firstCause = causes.get(0);
           for (LensException cause : causes) {
             if (!cause.equals(firstCause)) {
-              throw new LensMultiCauseException(ImmutableList.copyOf(causes));
+              throw new LensMultiCauseException(StringUtils.join(failureCauses, '\n'), ImmutableList.copyOf(causes));
             }
           }
           throw firstCause;
@@ -1555,11 +1555,15 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
       if (prepared != null) {
         destroyPreparedQuery(prepared);
       }
-      throw e;
-    } catch (UnsupportedEncodingException e) {
-      if (prepared != null) {
-        destroyPreparedQuery(prepared);
+      log.error("Explain and prepare failed", e);
+      QueryPlan plan;
+      if (e.getCause() != null && e.getCause().getMessage() != null) {
+        plan = new QueryPlan(true, e.getCause().getMessage());
+      } else {
+        plan = new QueryPlan(true, e.getMessage());
       }
+      return plan;
+    } catch (UnsupportedEncodingException e) {
       throw new LensException(e);
     } finally {
       release(sessionHandle);
@@ -2255,16 +2259,25 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
   public QueryPlan explain(final String requestId, LensSessionHandle sessionHandle, String query, LensConf lensConf)
     throws LensException {
     try {
-      log.info("Explain: session:{} query:{}", sessionHandle, query);
+      log.info("Explain: session:{} query:{}",  sessionHandle, query);
       acquire(sessionHandle);
       Configuration qconf = getLensConf(sessionHandle, lensConf);
-      ExplainQueryContext explainQueryContext = new ExplainQueryContext(requestId, query, getSession(sessionHandle)
-          .getLoggedInUser(), lensConf, qconf, drivers.values());
+      ExplainQueryContext explainQueryContext = new ExplainQueryContext(requestId, query,
+        getSession(sessionHandle).getLoggedInUser(), lensConf, qconf, drivers.values());
       explainQueryContext.setLensSessionIdentifier(sessionHandle.getPublicId().toString());
       accept(query, qconf, SubmitOp.EXPLAIN);
       rewriteAndSelect(explainQueryContext);
       addSessionResourcesToDriver(explainQueryContext);
       return explainQueryContext.getSelectedDriver().explain(explainQueryContext).toQueryPlan();
+    } catch (LensException e) {
+      log.error("Error during explain :", e);
+      QueryPlan plan;
+      if (e.getCause() != null && e.getCause().getMessage() != null) {
+        plan = new QueryPlan(true, e.getCause().getMessage());
+      } else {
+        plan = new QueryPlan(true, e.getMessage());
+      }
+      return plan;
     } catch (UnsupportedEncodingException e) {
       throw new LensException(e);
     } finally {

http://git-wip-us.apache.org/repos/asf/lens/blob/fb235faa/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 96f6aea..6bfd77a 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
@@ -77,7 +77,7 @@ public class QueryServiceResource {
 
   private void validateSessionId(final LensSessionHandle sessionHandle) throws LensException {
     if (sessionHandle == null) {
-      throw new LensException(SESSION_ID_NOT_PROVIDED.getLensErrorInfo());
+      throw new LensException(SESSION_ID_NOT_PROVIDED.getValue());
     }
   }
 
@@ -104,7 +104,7 @@ public class QueryServiceResource {
 
   private void validateQuery(String query) throws LensException {
     if (StringUtils.isBlank(query)) {
-      throw new LensException(NULL_OR_EMPTY_OR_BLANK_QUERY.getLensErrorInfo());
+      throw new LensException(NULL_OR_EMPTY_OR_BLANK_QUERY.getValue());
     }
   }
   /**
@@ -340,24 +340,19 @@ public class QueryServiceResource {
    * @return {@link QueryPrepareHandle} incase of {link org.apache.lens.api.query.SubmitOp#PREPARE} operation.
    *         {@link QueryPlan} incase of {@link org.apache.lens.api.query.SubmitOp#EXPLAIN_AND_PREPARE}
    *         and the query plan will contain the prepare handle as well.
-   * @throws LensException
    */
   @POST
   @Path("preparedqueries")
   @Consumes({MediaType.MULTIPART_FORM_DATA})
   @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, MediaType.TEXT_PLAIN})
   @MultiPurposeResource(formParamName = "operation")
-  public LensAPIResult<? extends QuerySubmitResult> prepareQuery(
-      @FormDataParam("sessionid") LensSessionHandle sessionid, @FormDataParam("query") String query,
-      @DefaultValue("") @FormDataParam("operation") String operation, @FormDataParam("conf") LensConf conf,
-      @DefaultValue("") @FormDataParam("queryName") String queryName) throws LensException {
-    final String requestId = this.logSegregationContext.getLogSegragationId();
-
+  public QuerySubmitResult prepareQuery(@FormDataParam("sessionid") LensSessionHandle sessionid,
+    @FormDataParam("query") String query, @DefaultValue("") @FormDataParam("operation") String operation,
+    @FormDataParam("conf") LensConf conf, @DefaultValue("") @FormDataParam("queryName") String queryName) {
     try {
       checkSessionId(sessionid);
       checkQuery(query);
       SubmitOp sop = null;
-      QuerySubmitResult result;
       try {
         sop = SubmitOp.valueOf(operation.toUpperCase());
       } catch (IllegalArgumentException e) {
@@ -368,20 +363,17 @@ public class QueryServiceResource {
       }
       switch (sop) {
       case PREPARE:
-        result = queryServer.prepare(sessionid, query, conf, queryName);
-        break;
+        return queryServer.prepare(sessionid, query, conf, queryName);
       case EXPLAIN_AND_PREPARE:
-        result = queryServer.explainAndPrepare(sessionid, query, conf, queryName);
-        break;
+        return queryServer.explainAndPrepare(sessionid, query, conf, queryName);
       default:
         throw new BadRequestException("Invalid operation type: " + operation + prepareClue);
       }
-      return LensAPIResult.composedOf(null, requestId, result);
     } catch (LensException e) {
-      e.buildLensErrorResponse(errorCollection, null, requestId);
-      throw e;
+      throw new WebApplicationException(e);
     }
   }
+
   /**
    * Destroy all the prepared queries; Can be filtered with user.
    *
@@ -398,11 +390,10 @@ public class QueryServiceResource {
    */
   @DELETE
   @Path("preparedqueries")
-  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, MediaType.TEXT_PLAIN })
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, MediaType.TEXT_PLAIN})
   public APIResult destroyPreparedQueries(@QueryParam("sessionid") LensSessionHandle sessionid,
-      @DefaultValue("") @QueryParam("user") String user, @DefaultValue("") @QueryParam("queryName") String queryName,
-      @DefaultValue("-1") @QueryParam("fromDate") long fromDate,
-      @DefaultValue("-1") @QueryParam("toDate") long toDate) {
+    @DefaultValue("") @QueryParam("user") String user, @DefaultValue("") @QueryParam("queryName") String queryName,
+    @DefaultValue("-1") @QueryParam("fromDate") long fromDate, @DefaultValue("-1") @QueryParam("toDate") long toDate) {
     checkSessionId(sessionid);
     int numDestroyed = 0;
     boolean failed = false;

http://git-wip-us.apache.org/repos/asf/lens/blob/fb235faa/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 4d4f137..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
@@ -49,7 +49,7 @@ public class ErrorResponseExpectedData {
 
     /* Assert Equal LensErrorTO (stack trace gets excluded in equality check) */
     final LensErrorTO actualLensErrorTO = lensAPIResult.getLensErrorTO();
-    assertEquals(actualLensErrorTO.getMessage(), expectedLensErrorTO.getMessage());
+    assertEquals(actualLensErrorTO, expectedLensErrorTO);
 
     /* Assert receipt of valid stacktraces */
     assertTrue(lensAPIResult.areValidStackTracesPresent(), "Received Lens Response:" + lensAPIResult);

http://git-wip-us.apache.org/repos/asf/lens/blob/fb235faa/lens-server/src/test/java/org/apache/lens/server/common/FailingQueryDriver.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/common/FailingQueryDriver.java b/lens-server/src/test/java/org/apache/lens/server/common/FailingQueryDriver.java
index 2f74ceb..64a75ce 100644
--- a/lens-server/src/test/java/org/apache/lens/server/common/FailingQueryDriver.java
+++ b/lens-server/src/test/java/org/apache/lens/server/common/FailingQueryDriver.java
@@ -34,7 +34,7 @@ public class FailingQueryDriver extends MockDriver {
     if (ctx.getUserQuery().contains("fail")) {
       return new FactPartitionBasedQueryCost(0.0);
     } else {
-      throw new LensException("Simulated Estimate Failure");
+      throw new LensException();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/fb235faa/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 18a8c8d..2189eb8 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
@@ -111,8 +111,8 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
     Response response = estimate(target(), Optional.<LensSessionHandle>absent(), Optional.of(MOCK_QUERY));
 
     final String expectedErrMsg = "Session id not provided. Please provide a session id.";
-    LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(
-        SESSION_ID_NOT_PROVIDED.getLensErrorInfo().getErrorCode(), expectedErrMsg, MOCK_STACK_TRACE);
+    LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(SESSION_ID_NOT_PROVIDED.getValue(), expectedErrMsg,
+      MOCK_STACK_TRACE);
     ErrorResponseExpectedData expectedData = new ErrorResponseExpectedData(BAD_REQUEST, expectedLensErrorTO);
 
     expectedData.verify(response);
@@ -126,8 +126,8 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
     Response response = estimate(target(), Optional.of(sessionId), testQuery);
 
     final String expectedErrMsg = "Query is not provided, or it is empty or blank. Please provide a valid query.";
-    LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(
-        NULL_OR_EMPTY_OR_BLANK_QUERY.getLensErrorInfo().getErrorCode(), expectedErrMsg, MOCK_STACK_TRACE);
+    LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(NULL_OR_EMPTY_OR_BLANK_QUERY.getValue(), expectedErrMsg,
+      MOCK_STACK_TRACE);
     ErrorResponseExpectedData expectedData = new ErrorResponseExpectedData(BAD_REQUEST, expectedLensErrorTO);
 
     expectedData.verify(response);
@@ -144,8 +144,7 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
     final String expectedErrMsg = "Provided Operation is not supported. Supported Operations are: "
       + "[estimate, execute, explain, execute_with_timeout]";
 
-    LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(
-        UNSUPPORTED_QUERY_SUBMIT_OPERATION.getLensErrorInfo().getErrorCode(),
+    LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(UNSUPPORTED_QUERY_SUBMIT_OPERATION.getValue(),
       expectedErrMsg, MOCK_STACK_TRACE, new SupportedQuerySubmitOperations());
     ErrorResponseExpectedData expectedData = new ErrorResponseExpectedData(BAD_REQUEST, expectedLensErrorTO);
 
@@ -166,9 +165,11 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
       expectedErrMsg, MOCK_STACK_TRACE);
     LensErrorTO childError2 = LensErrorTO.composedOf(INTERNAL_SERVER_ERROR.getValue(),
         expectedErrMsg, MOCK_STACK_TRACE);
+    LensErrorTO childError3 = LensErrorTO.composedOf(INTERNAL_SERVER_ERROR.getValue(),
+        expectedErrMsg, MOCK_STACK_TRACE);
 
     LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(INTERNAL_SERVER_ERROR.getValue(),
-        expectedErrMsg, MOCK_STACK_TRACE, Arrays.asList(childError1, childError2));
+        expectedErrMsg, MOCK_STACK_TRACE, Arrays.asList(childError1, childError2, childError3));
 
     ErrorResponseExpectedData expectedData = new ErrorResponseExpectedData(Status.INTERNAL_SERVER_ERROR,
       expectedLensErrorTO);
@@ -184,7 +185,7 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
     Response response = estimate(target(), Optional.of(sessionId), Optional.of(MOCK_QUERY));
 
     final String expectedErrMsg = "Syntax Error: line 1:0 cannot recognize input near 'mock' '-' 'query'";
-    LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(SYNTAX_ERROR.getLensErrorInfo().getErrorCode(),
+    LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(SYNTAX_ERROR.getValue(),
       expectedErrMsg, MOCK_STACK_TRACE);
     ErrorResponseExpectedData expectedData = new ErrorResponseExpectedData(BAD_REQUEST, expectedLensErrorTO);
 
@@ -289,9 +290,8 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
       final ColUnAvailableInTimeRange expectedErrorPayload = new ColUnAvailableInTimeRange(testDimensionField,
         expecAvailableFrom, expecAvailableTill);
 
-      LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(
-          COLUMN_UNAVAILABLE_IN_TIME_RANGE.getLensErrorInfo().getErrorCode(),
-          expectedErrMsg, MOCK_STACK_TRACE, expectedErrorPayload, null);
+      LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(COLUMN_UNAVAILABLE_IN_TIME_RANGE.getValue(),
+        expectedErrMsg, MOCK_STACK_TRACE, expectedErrorPayload, null);
       ErrorResponseExpectedData expectedData = new ErrorResponseExpectedData(BAD_REQUEST, expectedLensErrorTO);
 
       expectedData.verify(response);

http://git-wip-us.apache.org/repos/asf/lens/blob/fb235faa/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 fa8d6ee..b3f5d93 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
@@ -375,7 +375,7 @@ public class TestQueryService extends LensJerseyTest {
       MediaType.APPLICATION_XML_TYPE));
 
     final QueryPlan plan2 = ptarget.request().post(Entity.entity(mp2, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensAPIResult<QueryPlan>>() {}).getData();
+      QueryPlan.class);
     assertEquals(plan2.getTablesQueried().size(), 1);
     assertTrue(plan2.getTablesQueried().get(0).endsWith(TEST_TABLE.toLowerCase()));
     assertNotNull(plan2.getPrepareHandle());
@@ -387,41 +387,47 @@ public class TestQueryService extends LensJerseyTest {
    * Test explain failure.
    *
    * @throws InterruptedException the interrupted exception
-   * @throws UnsupportedEncodingException
    */
   @Test
-  public void testExplainFailure() throws InterruptedException, UnsupportedEncodingException {
+  public void testExplainFailure() throws InterruptedException {
     final WebTarget target = target().path("queryapi/queries");
 
     final FormDataMultiPart mp = new FormDataMultiPart();
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionid").build(), lensSessionId,
-        MediaType.APPLICATION_XML_TYPE));
-    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("query").build(), "select NO_ID from "
-        + TEST_TABLE));
+      MediaType.APPLICATION_XML_TYPE));
+    mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("query").build(),
+      "select NO_ID from " + TEST_TABLE));
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("operation").build(), "explain"));
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
-        MediaType.APPLICATION_XML_TYPE));
-
-    final Response responseExplain = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE));
+      MediaType.APPLICATION_XML_TYPE));
 
-    assertEquals(responseExplain.getStatus(), Response.Status.INTERNAL_SERVER_ERROR.getStatusCode());
+    final QueryPlan plan = target.request()
+      .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
+        new GenericType<LensAPIResult<QueryPlan>>() {}).getData();
+    assertTrue(plan.isError());
+    assertNotNull(plan.getErrorMsg());
+    assertTrue(plan.getErrorMsg()
+      .contains("Invalid table alias or column reference 'NO_ID': " + "(possible column names are: id, idstr)"));
 
     // Test explain and prepare
     final WebTarget ptarget = target().path("queryapi/preparedqueries");
 
     final FormDataMultiPart mp2 = new FormDataMultiPart();
     mp2.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionid").build(), lensSessionId,
-        MediaType.APPLICATION_XML_TYPE));
+      MediaType.APPLICATION_XML_TYPE));
     mp2.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("query").build(), "select NO_ID from "
-        + TEST_TABLE));
+      + TEST_TABLE));
     mp2.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("operation").build(), "explain_and_prepare"));
     mp2.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("conf").fileName("conf").build(), new LensConf(),
-        MediaType.APPLICATION_XML_TYPE));
-
-    final Response responseExplainAndPrepare = target.request().post(
-        Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE));
+      MediaType.APPLICATION_XML_TYPE));
 
-    assertEquals(responseExplainAndPrepare.getStatus(), Response.Status.INTERNAL_SERVER_ERROR.getStatusCode());
+    final QueryPlan plan2 = ptarget.request().post(Entity.entity(mp2, MediaType.MULTIPART_FORM_DATA_TYPE),
+      QueryPlan.class);
+    assertTrue(plan2.isError());
+    assertNotNull(plan2.getErrorMsg());
+    assertNull(plan2.getPrepareHandle());
+    assertTrue(plan2.getErrorMsg().contains("Invalid table alias or column reference 'NO_ID': "
+      + "(possible column names are: id, idstr)"));
   }
 
   // post to preparedqueries
@@ -451,7 +457,7 @@ public class TestQueryService extends LensJerseyTest {
       MediaType.APPLICATION_XML_TYPE));
 
     final QueryPrepareHandle pHandle = target.request().post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-        new GenericType<LensAPIResult<QueryPrepareHandle>>() {}).getData();
+      QueryPrepareHandle.class);
 
     // Get all prepared queries
     List<QueryPrepareHandle> allQueries = (List<QueryPrepareHandle>) target.queryParam("sessionid", lensSessionId)
@@ -549,9 +555,7 @@ public class TestQueryService extends LensJerseyTest {
       MediaType.APPLICATION_XML_TYPE));
 
     final QueryPlan plan = target.request()
-      .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
-          new GenericType<LensAPIResult<QueryPlan>>() {}).getData();
-
+      .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE), QueryPlan.class);
     assertEquals(plan.getTablesQueried().size(), 1);
     assertTrue(plan.getTablesQueried().get(0).endsWith(TEST_TABLE.toLowerCase()));
     assertNotNull(plan.getPrepareHandle());
@@ -1421,9 +1425,7 @@ public class TestQueryService extends LensJerseyTest {
     final Response response = target.request()
       .post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE));
 
-
-    LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(
-        LensCubeErrorCode.NEITHER_CUBE_NOR_DIMENSION.getLensErrorInfo().getErrorCode(),
+    LensErrorTO expectedLensErrorTO = LensErrorTO.composedOf(LensCubeErrorCode.NEITHER_CUBE_NOR_DIMENSION.getValue(),
       "Neither cube nor dimensions accessed in the query", TestDataUtils.MOCK_STACK_TRACE);
     ErrorResponseExpectedData expectedData = new ErrorResponseExpectedData(BAD_REQUEST, expectedLensErrorTO);