You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by sh...@apache.org on 2015/12/30 08:10:59 UTC

[40/50] [abbrv] lens git commit: LENS-270 : The exception thrown for no candidate fact should contain only brief error

LENS-270 : The exception thrown for no candidate fact should contain only brief error


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

Branch: refs/heads/LENS-581
Commit: 36166a2e58a2a89bd97dc8595cb7920fbf4253d8
Parents: bf4c0be
Author: Sushil Mohanty <su...@apache.org>
Authored: Mon Dec 14 09:57:45 2015 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Mon Dec 14 09:57:45 2015 +0530

----------------------------------------------------------------------
 .../NoCandidateFactAvailableException.java      | 48 ++++++++++++++++++++
 .../lens/cube/parse/CubeQueryContext.java       |  4 +-
 .../org/apache/lens/cube/parse/PruneCauses.java | 12 +++++
 .../lens/cube/parse/TestBaseCubeQueries.java    | 19 +++++++-
 .../lens/cube/parse/TestCubeRewriter.java       |  7 ++-
 .../cube/parse/TestDenormalizationResolver.java |  4 +-
 .../lens/cube/parse/TestTimeRangeResolver.java  |  4 +-
 7 files changed, 91 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/36166a2e/lens-cube/src/main/java/org/apache/lens/cube/error/NoCandidateFactAvailableException.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/error/NoCandidateFactAvailableException.java b/lens-cube/src/main/java/org/apache/lens/cube/error/NoCandidateFactAvailableException.java
new file mode 100644
index 0000000..b2568ff
--- /dev/null
+++ b/lens-cube/src/main/java/org/apache/lens/cube/error/NoCandidateFactAvailableException.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.cube.error;
+
+import org.apache.lens.cube.metadata.CubeFactTable;
+import org.apache.lens.cube.parse.PruneCauses;
+import org.apache.lens.server.api.error.LensException;
+
+
+public class NoCandidateFactAvailableException extends LensException {
+
+  private final PruneCauses<CubeFactTable> briefAndDetailedError;
+
+  public NoCandidateFactAvailableException(PruneCauses<CubeFactTable> briefAndDetailedError) {
+    super(LensCubeErrorCode.NO_CANDIDATE_FACT_AVAILABLE.getLensErrorInfo(), briefAndDetailedError.getBriefCause());
+    this.briefAndDetailedError = briefAndDetailedError;
+  }
+
+  public PruneCauses.BriefAndDetailedError getJsonMessage() {
+    return briefAndDetailedError.toJsonObject();
+  }
+
+  @Override
+  public int compareTo(LensException e) {
+    //Compare the max CandidateTablePruneCode coming from different instances.
+    if (e instanceof NoCandidateFactAvailableException) {
+      return briefAndDetailedError.getMaxCause().compareTo(
+               ((NoCandidateFactAvailableException) e).briefAndDetailedError.getMaxCause());
+    }
+    return super.compareTo(e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/36166a2e/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java
index cf114c9..f75a6b9 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java
@@ -32,6 +32,7 @@ import java.io.IOException;
 import java.util.*;
 
 import org.apache.lens.cube.error.LensCubeErrorCode;
+import org.apache.lens.cube.error.NoCandidateFactAvailableException;
 import org.apache.lens.cube.metadata.*;
 import org.apache.lens.cube.parse.CandidateTablePruneCause.CandidateTablePruneCode;
 import org.apache.lens.server.api.error.LensException;
@@ -845,7 +846,8 @@ public class CubeQueryContext implements TrackQueriedColumns {
             }
           }
         }
-        throw new LensException(LensCubeErrorCode.NO_CANDIDATE_FACT_AVAILABLE.getLensErrorInfo(), reason);
+        log.error("Query rewrite failed due to NO_CANDIDATE_FACT_AVAILABLE, Cause {}", factPruningMsgs.toJsonObject());
+        throw new NoCandidateFactAvailableException(factPruningMsgs);
       }
     }
     return facts;

http://git-wip-us.apache.org/repos/asf/lens/blob/36166a2e/lens-cube/src/main/java/org/apache/lens/cube/parse/PruneCauses.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/PruneCauses.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/PruneCauses.java
index 7a92b3b..9b5a52f 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/PruneCauses.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/PruneCauses.java
@@ -41,6 +41,8 @@ public class PruneCauses<T extends AbstractCubeTable> extends HashMap<T, List<Ca
   private final HashMap<CandidateTablePruneCause, List<T>> reversed = reverse();
   @Getter(lazy = true)
   private final HashMap<String, List<CandidateTablePruneCause>> compact = computeCompact();
+  @Getter(lazy = true)
+  private final CandidateTablePruneCode maxCause  = computeMaxCause();
 
   private HashMap<String, List<CandidateTablePruneCause>> computeCompact() {
     HashMap<String, List<CandidateTablePruneCause>> detailedMessage = Maps.newHashMap();
@@ -81,6 +83,16 @@ public class PruneCauses<T extends AbstractCubeTable> extends HashMap<T, List<Ca
     return new BriefAndDetailedError(getBriefCause(), getCompact());
   }
 
+  private CandidateTablePruneCode computeMaxCause() {
+    CandidateTablePruneCode maxCause = CandidateTablePruneCode.values()[0];
+    for (CandidateTablePruneCause cause : getReversed().keySet()) {
+      if (cause.getCause().compareTo(maxCause) > 0) {
+        maxCause = cause.getCause();
+      }
+    }
+    return maxCause;
+  }
+
   public String getBriefCause() {
     CandidateTablePruneCode maxCause = CandidateTablePruneCode.values()[0];
     for (CandidateTablePruneCause cause : getReversed().keySet()) {

http://git-wip-us.apache.org/repos/asf/lens/blob/36166a2e/lens-cube/src/test/java/org/apache/lens/cube/parse/TestBaseCubeQueries.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestBaseCubeQueries.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestBaseCubeQueries.java
index 97c6d08..a5886dc 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestBaseCubeQueries.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestBaseCubeQueries.java
@@ -34,6 +34,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.lens.cube.error.LensCubeErrorCode;
+import org.apache.lens.cube.error.NoCandidateFactAvailableException;
 import org.apache.lens.cube.metadata.TimeRange;
 import org.apache.lens.cube.metadata.UpdatePeriod;
 import org.apache.lens.cube.parse.CandidateTablePruneCause.CandidateTablePruneCode;
@@ -67,6 +68,18 @@ public class TestBaseCubeQueries extends TestQueryRewrite {
   }
 
   @Test
+  public void testNoCandidateFactAvailableExceptionCompareTo() throws Exception {
+    //maxCause : COLUMN_NOT_FOUND, Ordinal : 9
+    NoCandidateFactAvailableException ne1 =(NoCandidateFactAvailableException)
+            getLensExceptionInRewrite("select dim1, test_time_dim, msr3, msr13 from basecube where "
+            + TWO_DAYS_RANGE, conf);
+    //maxCause : FACT_NOT_AVAILABLE_IN_RANGE, Ordinal : 1
+    NoCandidateFactAvailableException ne2 = (NoCandidateFactAvailableException)
+            getLensExceptionInRewrite("cube select dim1 from " + cubeName + " where " + LAST_YEAR_RANGE, getConf());
+    assertEquals(ne1.compareTo(ne2), 8);
+  }
+
+  @Test
   public void testColumnErrors() throws Exception {
     LensException e;
 
@@ -78,7 +91,8 @@ public class TestBaseCubeQueries extends TestQueryRewrite {
       + TWO_DAYS_RANGE, conf);
     assertEquals(e.getErrorCode(),
         LensCubeErrorCode.NO_CANDIDATE_FACT_AVAILABLE.getLensErrorInfo().getErrorCode());
-    PruneCauses.BriefAndDetailedError pruneCauses = extractPruneCause(e);
+    NoCandidateFactAvailableException ne = (NoCandidateFactAvailableException) e;
+    PruneCauses.BriefAndDetailedError pruneCauses = ne.getJsonMessage();
     String regexp = String.format(CandidateTablePruneCause.CandidateTablePruneCode.COLUMN_NOT_FOUND.errorFormat,
       "Column Sets: (.*?)", "queriable together");
     Matcher matcher = Pattern.compile(regexp).matcher(pruneCauses.getBrief());
@@ -494,7 +508,8 @@ public class TestBaseCubeQueries extends TestQueryRewrite {
     conf.setBoolean(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, true);
     LensException exc =
       getLensExceptionInRewrite("cube select msr12 from basecube where " + TWO_DAYS_RANGE, conf);
-    PruneCauses.BriefAndDetailedError pruneCause = extractPruneCause(exc);
+    NoCandidateFactAvailableException ne = (NoCandidateFactAvailableException) exc;
+    PruneCauses.BriefAndDetailedError pruneCause = ne.getJsonMessage();
     assertTrue(pruneCause.getBrief().contains("Missing partitions"));
     assertEquals(pruneCause.getDetails().get("testfact2_base").iterator().next().getCause(), MISSING_PARTITIONS);
     assertEquals(pruneCause.getDetails().get("testfact2_base").iterator().next().getMissingPartitions().size(), 1);

http://git-wip-us.apache.org/repos/asf/lens/blob/36166a2e/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java
index 0f05556..802ff42 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java
@@ -32,6 +32,7 @@ import java.text.SimpleDateFormat;
 import java.util.*;
 
 import org.apache.lens.cube.error.LensCubeErrorCode;
+import org.apache.lens.cube.error.NoCandidateFactAvailableException;
 import org.apache.lens.cube.metadata.*;
 import org.apache.lens.cube.parse.CandidateTablePruneCause.SkipStorageCause;
 import org.apache.lens.cube.parse.CandidateTablePruneCause.SkipStorageCode;
@@ -186,7 +187,8 @@ public class TestCubeRewriter extends TestQueryRewrite {
     LensException th = getLensExceptionInRewrite(
       "select SUM(msr2) from testCube" + " where " + TWO_DAYS_RANGE, conf);
     assertEquals(th.getErrorCode(), LensCubeErrorCode.NO_CANDIDATE_FACT_AVAILABLE.getLensErrorInfo().getErrorCode());
-    PruneCauses.BriefAndDetailedError pruneCauses = extractPruneCause(th);
+    NoCandidateFactAvailableException ne = (NoCandidateFactAvailableException) th;
+    PruneCauses.BriefAndDetailedError pruneCauses = ne.getJsonMessage();
     int endIndex = MISSING_PARTITIONS.errorFormat.length() - 3;
     assertEquals(
       pruneCauses.getBrief().substring(0, endIndex),
@@ -1031,7 +1033,8 @@ public class TestCubeRewriter extends TestQueryRewrite {
     LensException e = getLensExceptionInRewrite(
       "select SUM(msr2) from testCube" + " where " + TWO_MONTHS_RANGE_UPTO_HOURS, conf);
     assertEquals(e.getErrorCode(), LensCubeErrorCode.NO_CANDIDATE_FACT_AVAILABLE.getLensErrorInfo().getErrorCode());
-    PruneCauses.BriefAndDetailedError pruneCauses = extractPruneCause(e);
+    NoCandidateFactAvailableException ne = (NoCandidateFactAvailableException) e;
+    PruneCauses.BriefAndDetailedError pruneCauses = ne.getJsonMessage();
 
     assertEquals(
       pruneCauses.getBrief().substring(0, MISSING_PARTITIONS.errorFormat.length() - 3),

http://git-wip-us.apache.org/repos/asf/lens/blob/36166a2e/lens-cube/src/test/java/org/apache/lens/cube/parse/TestDenormalizationResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestDenormalizationResolver.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestDenormalizationResolver.java
index 36c1dba..af9daad 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestDenormalizationResolver.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestDenormalizationResolver.java
@@ -24,6 +24,7 @@ import static org.apache.lens.cube.parse.CubeTestSetup.*;
 
 import java.util.*;
 
+import org.apache.lens.cube.error.NoCandidateFactAvailableException;
 import org.apache.lens.cube.parse.CandidateTablePruneCause.CandidateTablePruneCode;
 import org.apache.lens.server.api.error.LensException;
 
@@ -149,7 +150,8 @@ public class TestDenormalizationResolver extends TestQueryRewrite {
     TestCubeRewriter.compareQueries(hqlQuery, expected);
     LensException e = getLensExceptionInRewrite(
       "select dim2big2, max(msr3)," + " msr2 from testCube" + " where " + TWO_DAYS_RANGE, tconf);
-    PruneCauses.BriefAndDetailedError error = extractPruneCause(e);
+    NoCandidateFactAvailableException ne = (NoCandidateFactAvailableException) e;
+    PruneCauses.BriefAndDetailedError error = ne.getJsonMessage();
     Assert.assertEquals(error.getBrief(), CandidateTablePruneCode.NO_CANDIDATE_STORAGES.errorFormat);
 
     HashMap<String, List<CandidateTablePruneCause>> details = error.getDetails();

http://git-wip-us.apache.org/repos/asf/lens/blob/36166a2e/lens-cube/src/test/java/org/apache/lens/cube/parse/TestTimeRangeResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestTimeRangeResolver.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestTimeRangeResolver.java
index da0e4f4..2ac837d 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestTimeRangeResolver.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestTimeRangeResolver.java
@@ -29,6 +29,7 @@ import static org.testng.Assert.assertTrue;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.lens.cube.error.NoCandidateFactAvailableException;
 import org.apache.lens.server.api.error.LensException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -66,7 +67,8 @@ public class TestTimeRangeResolver extends TestQueryRewrite {
     LensException e =
       getLensExceptionInRewrite("cube select msr2 from " + cubeName + " where " + LAST_YEAR_RANGE,
         getConf());
-    PruneCauses.BriefAndDetailedError causes = extractPruneCause(e);
+    NoCandidateFactAvailableException ne = (NoCandidateFactAvailableException) e;
+    PruneCauses.BriefAndDetailedError causes = ne.getJsonMessage();
     assertTrue(causes.getBrief().contains("Columns [msr2] are not present in any table"));
     assertEquals(causes.getDetails().size(), 2);