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 2018/02/06 05:46:17 UTC

[01/50] lens git commit: LENS-1431 Data-availability feature fixes

Repository: lens
Updated Branches:
  refs/heads/master d0b78e81b -> 9e7012ed9


LENS-1431 Data-availability feature fixes


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

Branch: refs/heads/master
Commit: 419e1904ec971ff1c944f7543ea6bfc6607555e7
Parents: 60e1277
Author: Sushrut Meghshyam Ikhar <su...@gmail.com>
Authored: Tue Jun 6 17:53:27 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Sat Jun 10 13:30:01 2017 +0800

----------------------------------------------------------------------
 .../lens/cube/parse/ExpressionResolver.java     |  4 --
 .../lens/cube/parse/StorageCandidate.java       |  8 +++-
 .../cube/parse/MockCompletenessChecker.java     | 10 ++--
 .../lens/cube/parse/TestCubeRewriter.java       | 49 +++++++++++++++++++-
 .../resources/schema/cubes/base/basecube.xml    |  5 ++
 .../resources/schema/cubes/derived/der1.xml     |  1 +
 .../schema/facts/testfact2_raw_base.xml         |  1 +
 .../schema/facts/virtual/virtualfact.xml        | 27 +++++++++++
 .../test/resources/schema/facts/virtualfact.xml | 27 -----------
 9 files changed, 95 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/419e1904/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
index ea6d5c7..b1654d1 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
@@ -556,12 +556,8 @@ class ExpressionResolver implements ContextRewriter {
               log.info("Removing expression {} as all tables have non reachable fields", esc);
               iterator.remove();
               removedEsc.add(esc);
-              removed = true;
               break;
             }
-            if (removed) {
-              continue;
-            }
             //remove expressions which are not valid in the timerange queried
             // If an expression is defined as
             // ex = a + b // from t1 to t2;

http://git-wip-us.apache.org/repos/asf/lens/blob/419e1904/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
index 3f5e2e5..7980797 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
@@ -651,6 +651,10 @@ public class StorageCandidate implements Candidate, CandidateTable {
   }
 
   private boolean evaluateMeasuresCompleteness(TimeRange timeRange) throws LensException {
+    if (getCubeMetastoreClient() == null || !getCubeMetastoreClient().isDataCompletenessCheckEnabled()) {
+      log.info("Skipping availability check for the fact table: {} as dataCompleteness check is not enabled", fact);
+      return true;
+    }
     String factDataCompletenessTag = fact.getDataCompletenessTag();
     if (factDataCompletenessTag == null) {
       log.info("Not checking completeness for the fact table:{} as the dataCompletenessTag is not set", fact);
@@ -673,7 +677,8 @@ public class StorageCandidate implements Candidate, CandidateTable {
       log.info("No Queried measures with the dataCompletenessTag, hence skipping the availability check");
       return true;
     }
-    boolean isDataComplete = false;
+    // default completenessTag will be true
+    boolean isDataComplete = true;
     DataCompletenessChecker completenessChecker = getCubeMetastoreClient().getCompletenessChecker();
     DateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
     formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
@@ -695,6 +700,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
             String measureorExprFromTag = tagToMeasureOrExprMap.get(tag);
             dataCompletenessMap.computeIfAbsent(measureorExprFromTag, k -> new HashMap<>())
               .put(formatter.format(completenessResult.getKey()), completenessResult.getValue());
+            // set completeness to false if availability for measure is below threshold
             isDataComplete = false;
           }
         }

http://git-wip-us.apache.org/repos/asf/lens/blob/419e1904/lens-cube/src/test/java/org/apache/lens/cube/parse/MockCompletenessChecker.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/MockCompletenessChecker.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/MockCompletenessChecker.java
index 76e81d5..c1936eb 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/MockCompletenessChecker.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/MockCompletenessChecker.java
@@ -33,10 +33,12 @@ public class MockCompletenessChecker implements DataCompletenessChecker {
     Calendar cal = Calendar.getInstance();
     cal.setTimeZone(TimeZone.getTimeZone("GMT"));
     cal.add(Calendar.DATE, -1);
-    if (factTag.equals("f1")) {
-      partitionCompleteness.put(cal.getTime(), 80f);
-    } else {
-      partitionCompleteness.put(cal.getTime(), 90f);
+    if (start.before(cal.getTime()) && end.after(cal.getTime())) {
+      if (factTag.equals("f1")) {
+        partitionCompleteness.put(cal.getTime(), 80f);
+      } else {
+        partitionCompleteness.put(cal.getTime(), 90f);
+      }
     }
     result.put("tag1", partitionCompleteness);
     return result;

http://git-wip-us.apache.org/repos/asf/lens/blob/419e1904/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 65ca861..d8c7335 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
@@ -981,7 +981,7 @@ public class TestCubeRewriter extends TestQueryRewrite {
     conf.setStrings(CubeQueryConfUtil.COMPLETENESS_CHECK_PART_COL, "dt");
     String hqlQuery = rewrite("select SUM(msr9) from basecube where " + TWO_DAYS_RANGE, conf);
     String expected = getExpectedQuery("basecube", "select sum(basecube.msr9) as `sum(msr9)` FROM ", null, null,
-        getWhereForHourly2days("basecube", "c1_testfact5_raw_base"));
+      getWhereForHourly2days("basecube", "c1_testfact5_raw_base"));
     compareQueries(hqlQuery, expected);
   }
 
@@ -1007,6 +1007,53 @@ public class TestCubeRewriter extends TestQueryRewrite {
     }
   }
 
+  /*
+   * The test is to check that query is rewritten successfully if there is missing entry in
+   * dataavailability service for the measure's tag
+   */
+  @Test
+  public void testQueryWithMeasureWithDataCompletenessTagWithDataAvailiability() throws ParseException,
+    LensException {
+    NoCandidateFactAvailableException ne;
+    PruneCauses.BriefAndDetailedError pruneCauses;
+    String hqlQuery;
+    String expected;
+
+    Configuration conf = getConf();
+    conf.setStrings(CubeQueryConfUtil.COMPLETENESS_CHECK_PART_COL, "dt");
+
+    conf.setBoolean(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, true);
+
+    // 1. data completeness check applicable
+    ne = getLensExceptionInRewrite("select SUM(msr16) from basecube" + " where " + TWO_DAYS_RANGE, conf);
+    pruneCauses = ne.getJsonMessage();
+    assertEquals(pruneCauses.getBrief().substring(0, 10), INCOMPLETE_PARTITION.errorFormat.substring(0, 10),
+      pruneCauses.getBrief());
+
+    // 2. time outside data completeness check but missing partitions
+    ne = getLensExceptionInRewrite("select SUM(msr16) from basecube where " + TWO_DAYS_RANGE_BEFORE_4_DAYS, conf);
+    pruneCauses = ne.getJsonMessage();
+    assertEquals(pruneCauses.getBrief().substring(0, 10), MISSING_PARTITIONS.errorFormat.substring(0, 10),
+      pruneCauses.getBrief());
+
+
+    conf.setBoolean(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, false);
+
+    // 3. query allowed on partial data although data incomplete
+    hqlQuery = rewrite("select SUM(msr16) from basecube" + " where " + TWO_DAYS_RANGE, conf);
+    expected = getExpectedQuery("basecube", "select sum(basecube.msr16)  as `sum(msr16)` FROM ", null, null,
+      getWhereForHourly2days("basecube", "c1_testfact2_raw_base"));
+    compareQueries(hqlQuery.toLowerCase(), expected.toLowerCase());
+
+    // 4. query allowed on partial data with missing partitions but outside data availability window
+    hqlQuery = rewrite("select SUM(msr16) from basecube" + " where " + TWO_DAYS_RANGE_BEFORE_4_DAYS, conf);
+    expected = getExpectedQuery("basecube", "select sum(basecube.msr16)  as `sum(msr16)` FROM ", null, null,
+      getWhereForUpdatePeriods("basecube", "c1_testfact2_raw_base",
+        DateUtils.addHours(getDateWithOffset(UpdatePeriod.DAILY, -6), -1),
+        getDateWithOffset(UpdatePeriod.DAILY, -4), Sets.newHashSet(UpdatePeriod.HOURLY)));
+    compareQueries(hqlQuery.toLowerCase(), expected.toLowerCase());
+  }
+
   @Test
   public void testCubeWhereQueryForMonthWithNoPartialData() throws Exception {
     Configuration conf = getConf();

http://git-wip-us.apache.org/repos/asf/lens/blob/419e1904/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/cubes/base/basecube.xml b/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
index 708b510..bcea938 100644
--- a/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
+++ b/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
@@ -90,6 +90,11 @@
     </measure>
     <measure _type="INT" name="segmsr1" default_aggr="SUM" description="segmentation measure 1"
              display_string="segmentation measure 1"/>
+    <measure _type="BIGINT" name="msr16" description="sixteenth measure">
+      <tags>
+        <property name="cube.measure.datacompleteness.tag" value="tag1"/>
+      </tags>
+    </measure>
   </measures>
   <dim_attributes>
     <dim_attribute _type="string" name="business" display_string="business unit"

http://git-wip-us.apache.org/repos/asf/lens/blob/419e1904/lens-cube/src/test/resources/schema/cubes/derived/der1.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/cubes/derived/der1.xml b/lens-cube/src/test/resources/schema/cubes/derived/der1.xml
index cf36cc7..01bec03 100644
--- a/lens-cube/src/test/resources/schema/cubes/derived/der1.xml
+++ b/lens-cube/src/test/resources/schema/cubes/derived/der1.xml
@@ -33,6 +33,7 @@
     <measure_name>msr1</measure_name>
     <measure_name>msr11</measure_name>
     <measure_name>msr9</measure_name>
+    <measure_name>msr16</measure_name>
   </measure_names>
   <dim_attr_names>
     <attr_name>dim1</attr_name>

http://git-wip-us.apache.org/repos/asf/lens/blob/419e1904/lens-cube/src/test/resources/schema/facts/testfact2_raw_base.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/testfact2_raw_base.xml b/lens-cube/src/test/resources/schema/facts/testfact2_raw_base.xml
index 0bab461..69fd8d9 100644
--- a/lens-cube/src/test/resources/schema/facts/testfact2_raw_base.xml
+++ b/lens-cube/src/test/resources/schema/facts/testfact2_raw_base.xml
@@ -24,6 +24,7 @@
     <column name="msr11" _type="int" comment="first measure"/>
     <column name="msr12" _type="float" comment="second measure"/>
     <column name="msr9" _type="bigint" comment="ninth measure"/>
+    <column name="msr16" _type="bigint" comment="sixteenth measure"/>
     <column name="d_time" _type="timestamp" comment="event time"/>
     <column name="processing_time" _type="timestamp" comment="processing time"/>
     <column name="dim1" _type="string" comment="base dim"/>

http://git-wip-us.apache.org/repos/asf/lens/blob/419e1904/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml b/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml
new file mode 100644
index 0000000..860965f
--- /dev/null
+++ b/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml
@@ -0,0 +1,27 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+  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.
+
+-->
+<x_virtual_fact_table source_fact_name="summary1" cube_name="virtualcube" name="virtualfact1" xmlns="uri:lens:cube:0.1"
+                      xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="uri:lens:cube:0.1 cube-0.1.xsd ">
+  <properties>
+    <property name="cube.fact.query.where.filter" value=" dim1 = 10 "/>
+  </properties>
+</x_virtual_fact_table>

http://git-wip-us.apache.org/repos/asf/lens/blob/419e1904/lens-cube/src/test/resources/schema/facts/virtualfact.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/virtualfact.xml b/lens-cube/src/test/resources/schema/facts/virtualfact.xml
deleted file mode 100644
index 860965f..0000000
--- a/lens-cube/src/test/resources/schema/facts/virtualfact.xml
+++ /dev/null
@@ -1,27 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-
-  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.
-
--->
-<x_virtual_fact_table source_fact_name="summary1" cube_name="virtualcube" name="virtualfact1" xmlns="uri:lens:cube:0.1"
-                      xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="uri:lens:cube:0.1 cube-0.1.xsd ">
-  <properties>
-    <property name="cube.fact.query.where.filter" value=" dim1 = 10 "/>
-  </properties>
-</x_virtual_fact_table>


[34/50] lens git commit: LENS-1465: Sync queries blocking threads

Posted by ra...@apache.org.
LENS-1465: Sync queries blocking threads


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

Branch: refs/heads/master
Commit: 13cbc811342378d8fb39f4d03be08145fbc4f986
Parents: 1d66797
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Wed Aug 30 16:10:13 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Thu Oct 5 11:13:09 2017 +0800

----------------------------------------------------------------------
 .../org/apache/lens/server/api/query/QueryContext.java   | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/13cbc811/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 466fb25..8176f03 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
@@ -198,7 +198,7 @@ public class QueryContext extends AbstractQueryContext implements FailureContext
   @Getter
   @Setter
   private transient Future queryLauncher;
-  transient List<QueryDriverStatusUpdateListener> driverStatusUpdateListeners = Lists.newArrayList();
+  transient List<QueryDriverStatusUpdateListener> driverStatusUpdateListeners = Lists.newCopyOnWriteArrayList();
   @Getter
   @Setter
   List<FailedAttempt> failedAttempts = Lists.newArrayList();
@@ -572,10 +572,9 @@ public class QueryContext extends AbstractQueryContext implements FailureContext
       getDriverStatus().setStatusMessage("Query " + getQueryHandleString() + " " + state.name().toLowerCase());
     }
     getDriverStatus().setState(state);
-    synchronized (this) {
-      for (QueryDriverStatusUpdateListener listener : this.driverStatusUpdateListeners) {
-        listener.onDriverStatusUpdated(getQueryHandle(), getDriverStatus());
-      }
+
+    for (QueryDriverStatusUpdateListener listener : this.driverStatusUpdateListeners) {
+      listener.onDriverStatusUpdated(getQueryHandle(), getDriverStatus());
     }
   }
 
@@ -588,7 +587,7 @@ public class QueryContext extends AbstractQueryContext implements FailureContext
   }
 
 
-  public synchronized void registerStatusUpdateListener(QueryDriverStatusUpdateListener driverStatusUpdateListener) {
+  public void registerStatusUpdateListener(QueryDriverStatusUpdateListener driverStatusUpdateListener) {
     this.driverStatusUpdateListeners.add(driverStatusUpdateListener);
   }
 


[12/50] lens git commit: LENS-1437 : Missing unit test cases for virtual fact

Posted by ra...@apache.org.
LENS-1437 : Missing unit test cases for virtual fact


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

Branch: refs/heads/master
Commit: fbad350798a342a4de5e5b8b566a6f055338fde3
Parents: d49f45a
Author: Rajitha R <ra...@gmail.com>
Authored: Tue Jun 27 16:12:56 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:42:54 2017 +0530

----------------------------------------------------------------------
 .../apache/lens/cube/parse/CubeTestSetup.java   |   4 +-
 .../lens/cube/parse/TestCubeRewriter.java       |  18 +--
 .../lens/cube/parse/TestVirtualFactQueries.java | 123 +++++++++++++++++++
 .../resources/schema/cubes/base/testcube2.xml   |  31 +++++
 .../resources/schema/cubes/base/virtualcube.xml |  23 +++-
 .../resources/schema/facts/testfact7_base.xml   |  67 ++++++++++
 .../resources/schema/facts/testfact8_base.xml   |  58 +++++++++
 .../resources/schema/facts/testfact9_base.xml   |  57 +++++++++
 .../schema/facts/virtual/virtualfact.xml        |   2 +-
 9 files changed, 361 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/fbad3507/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
index 48869c2..0366e56 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
@@ -364,7 +364,7 @@ public class CubeTestSetup {
   // storageName[0] is hourly
   // storageName[1] is daily
   // storageName[2] is monthly
-  public static Map<String, String> getWhereForMonthlyDailyAndHourly2months(String... storageTables) {
+  public static Map<String, String> getWhereForMonthlyDailyAndHourly2months(String cubeName, String... storageTables) {
     Map<String, String> storageTableToWhereClause = new LinkedHashMap<String, String>();
     List<String> hourlyparts = new ArrayList<String>();
     List<String> dailyparts = new ArrayList<String>();
@@ -411,7 +411,7 @@ public class CubeTestSetup {
       tables.append(storageTables[0]);
     }
     Collections.sort(parts);
-    storageTableToWhereClause.put(tables.toString(), StorageUtil.getWherePartClause("dt", TEST_CUBE_NAME, parts));
+    storageTableToWhereClause.put(tables.toString(), StorageUtil.getWherePartClause("dt", cubeName, parts));
     return storageTableToWhereClause;
   }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/fbad3507/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 d8c7335..1c75e6c 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
@@ -145,22 +145,6 @@ public class TestCubeRewriter extends TestQueryRewrite {
 //    assertNotNull(rewrittenQuery.getNonExistingParts());
   }
 
-
-  @Test
-  public void testVirtualFactCubeSimpleQuery() throws Exception {
-    Configuration conf = getConf();
-    conf.set(DRIVER_SUPPORTED_STORAGES, "C1");
-    CubeQueryContext rewrittenQuery =
-      rewriteCtx("select SUM(msr2) from virtualCube where " + TWO_DAYS_RANGE, getConfWithStorages("C1"));
-    String expected = getExpectedQuery(VIRTUAL_CUBE_NAME, "select sum(virtualcube.msr2) as `sum(msr2)` FROM ",
-      null, "AND ( dim1 = 10 )", getWhereForDailyAndHourly2days(VIRTUAL_CUBE_NAME,
-        "C1_summary1"));
-    String hql = rewrittenQuery.toHQL();
-    compareQueries(hql, expected);
-    System.out.println("Non existing parts:" + rewrittenQuery.getNonExistingParts());
-  }
-
-
   @Test
   public void testMaxCoveringFact() throws Exception {
     Configuration conf = getConf();
@@ -954,7 +938,7 @@ public class TestCubeRewriter extends TestQueryRewrite {
       rewrite("select SUM(msr2) from testCube" + " where " + TWO_MONTHS_RANGE_UPTO_HOURS, getConfWithStorages("C2"));
     String expected =
       getExpectedQuery(TEST_CUBE_NAME, "select sum(testcube.msr2) as `sum(msr2)` FROM ", null, null,
-        getWhereForMonthlyDailyAndHourly2months("C2_testfact"));
+        getWhereForMonthlyDailyAndHourly2months(TEST_CUBE_NAME, "C2_testfact"));
     compareQueries(hqlQuery, expected);
   }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/fbad3507/lens-cube/src/test/java/org/apache/lens/cube/parse/TestVirtualFactQueries.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestVirtualFactQueries.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestVirtualFactQueries.java
new file mode 100644
index 0000000..4bcb376
--- /dev/null
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestVirtualFactQueries.java
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.lens.cube.parse;
+
+import static org.apache.lens.cube.metadata.DateFactory.*;
+import static org.apache.lens.cube.parse.CubeQueryConfUtil.*;
+import static org.apache.lens.cube.parse.CubeQueryConfUtil.DISABLE_AGGREGATE_RESOLVER;
+import static org.apache.lens.cube.parse.CubeTestSetup.*;
+
+import static org.testng.Assert.assertEquals;
+
+import org.apache.lens.server.api.LensServerAPITestUtil;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.testng.annotations.BeforeTest;
+import org.testng.annotations.Test;
+
+public class TestVirtualFactQueries extends TestQueryRewrite {
+
+  private Configuration conf;
+
+  @BeforeTest
+  public void setupDriver() throws Exception {
+    conf = LensServerAPITestUtil.getConfiguration(
+      DRIVER_SUPPORTED_STORAGES, "C1",
+      DISABLE_AUTO_JOINS, false,
+      ENABLE_SELECT_TO_GROUPBY, true,
+      ENABLE_GROUP_BY_TO_SELECT, true,
+      DISABLE_AGGREGATE_RESOLVER, false);
+  }
+
+  @Override
+  public Configuration getConf() {
+    return new Configuration(conf);
+  }
+
+  @Test
+  public void testVirtualFactDayQuery() throws Exception {
+    CubeQueryContext rewrittenQuery =
+      rewriteCtx("select SUM(msr1) from virtualCube where " + TWO_DAYS_RANGE, getConfWithStorages("C1"));
+    String expected = getExpectedQuery(VIRTUAL_CUBE_NAME, "select sum(virtualcube.msr1) as `sum(msr1)` FROM ",
+      null, "AND ( dim1 = 10 )", getWhereForDailyAndHourly2days(VIRTUAL_CUBE_NAME,
+        "C1_testfact9_base"));
+    String hql = rewrittenQuery.toHQL();
+    compareQueries(hql, expected);
+  }
+
+  @Test
+  public void testVirtualFactMonthQuery() throws Exception {
+
+    CubeQueryContext rewrittenQuery =
+      rewriteCtx("select SUM(msr1) from virtualCube where " + TWO_MONTHS_RANGE_UPTO_HOURS, getConfWithStorages("C1"));
+    String expected = getExpectedQuery(VIRTUAL_CUBE_NAME, "select sum(virtualcube.msr1) as `sum(msr1)` FROM ",
+      null, "AND ( dim1 = 10 )", getWhereForMonthlyDailyAndHourly2months("virtualcube",
+        "C1_testfact9_base"));
+    String hql = rewrittenQuery.toHQL();
+    compareQueries(hql, expected);
+  }
+
+  static void compareQueries(String actual, String expected) {
+    assertEquals(new TestQuery(actual), new TestQuery(expected));
+  }
+
+  @Test
+  public void testVirtualFactUnionQuery() throws Exception {
+
+    String expectedInnerSelect = getExpectedQuery("virtualcube", "SELECT (virtualcube.cityid) AS `alias0`,"
+        + " sum((virtualcube.msr2)) AS `alias1`,0.0 AS `alias2` FROM ",
+      null, null, "GROUP BY (virtualcube.cityid)", null,
+      getWhereForDailyAndHourly2days("virtualcube", "c1_testfact8_base"))
+      + " UNION ALL " + getExpectedQuery("virtualcube", "SELECT (virtualcube.cityid) AS `alias0`,"
+        + "0.0 AS `alias1`, sum((virtualcube.msr3)) AS `alias2` FROM ",
+      null, null, "GROUP BY (virtualcube.cityid)", null,
+      getWhereForDailyAndHourly2days("virtualcube", "c1_testfact7_base"));
+
+
+    String expected = "SELECT (virtualcube.alias0) AS `cityid`,"
+      + " sum((virtualcube.alias1)) AS `sum(msr2)`, sum((virtualcube.alias2)) AS `sum(msr3)`"
+      + " FROM (" + expectedInnerSelect + ") AS virtualcube GROUP BY (virtualcube.alias0)";
+
+    CubeQueryContext rewrittenQuery =
+      rewriteCtx("select cityid as `cityid`, SUM(msr2), SUM(msr3) from virtualcube where " + TWO_DAYS_RANGE,
+        getConfWithStorages("C1"));
+    String hql = rewrittenQuery.toHQL();
+    compareQueries(hql, expected);
+  }
+
+  @Test
+  public void testVirtualFactJoinQuery() throws Exception {
+    String query, hqlQuery, expected;
+
+    // Single joinchain with direct link
+    query = "select cubestate.name, sum(msr2) from virtualcube where " + TWO_DAYS_RANGE + " group by cubestate.name";
+    hqlQuery = rewrite(query, conf);
+    expected = getExpectedQuery("virtualcube", "SELECT (cubestate.name) as `name`, sum((virtualcube.msr2)) "
+        + "as `sum(msr2)` FROM ",
+      " join " + getDbName() + "c1_statetable cubestate ON virtualcube.stateid=cubeState.id and cubeState.dt= 'latest'",
+      null, "group by cubestate.name",
+      null, getWhereForDailyAndHourly2days("virtualcube", "c1_testfact8_base"));
+    TestCubeRewriter.compareQueries(hqlQuery, expected);
+
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/lens/blob/fbad3507/lens-cube/src/test/resources/schema/cubes/base/testcube2.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/cubes/base/testcube2.xml b/lens-cube/src/test/resources/schema/cubes/base/testcube2.xml
new file mode 100644
index 0000000..9740e00
--- /dev/null
+++ b/lens-cube/src/test/resources/schema/cubes/base/testcube2.xml
@@ -0,0 +1,31 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+
+  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.
+
+-->
+<x_base_cube name="testcube2" xmlns="uri:lens:cube:0.1">
+  <properties>
+    <property name="cube.timedim.partition.d_time" value="dt"/>
+    <property name="cube.testcube.timed.dimensions.list" value="d_time"/>
+  </properties>
+  <measures>
+    <measure _type="FLOAT" default_aggr="SUM" unit="RS" name="msr1" display_string="Measure1"
+             description="first measure"/>
+  </measures>
+</x_base_cube>

http://git-wip-us.apache.org/repos/asf/lens/blob/fbad3507/lens-cube/src/test/resources/schema/cubes/base/virtualcube.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/cubes/base/virtualcube.xml b/lens-cube/src/test/resources/schema/cubes/base/virtualcube.xml
index 2a5e0b0..187ac29 100644
--- a/lens-cube/src/test/resources/schema/cubes/base/virtualcube.xml
+++ b/lens-cube/src/test/resources/schema/cubes/base/virtualcube.xml
@@ -25,12 +25,31 @@
     <property name="cube.virtualcube.timed.dimensions.list" value="d_time"/>
   </properties>
   <measures>
+    <measure _type="FLOAT" default_aggr="SUM" unit="RS" name="msr1" display_string="Measure1"
+             description="first measure"/>
     <measure _type="FLOAT" default_aggr="SUM" unit="RS" name="msr2" display_string="Measure2"
              description="second measure"/>
+    <measure _type="DOUBLE" default_aggr="SUM" unit="RS" name="msr3" display_string="Measure3"
+             description="third measure"/>
 
   </measures>
   <dim_attributes>
-    <dim_attribute _type="string" name="dim1" description="basedim">
-    </dim_attribute>
+    <dim_attribute _type="string" name="dim1" description="basedim"/>
+    <dim_attribute _type="int" name="cityid" description="basedim"/>
+    <dim_attribute name="stateid" _type="int" description="state id"/>
   </dim_attributes>
+  <join_chains>
+    <join_chain dest_table="statedim" name="cubestate" display_string="cube-state" description="state thru cube">
+      <paths>
+        <path>
+          <edges>
+            <edge>
+              <from table="virtualcube" column="stateid" maps_to_many="false"/>
+              <to table="statedim" column="id" maps_to_many="false"/>
+            </edge>
+          </edges>
+        </path>
+      </paths>
+    </join_chain>
+  </join_chains>
 </x_base_cube>

http://git-wip-us.apache.org/repos/asf/lens/blob/fbad3507/lens-cube/src/test/resources/schema/facts/testfact7_base.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/testfact7_base.xml b/lens-cube/src/test/resources/schema/facts/testfact7_base.xml
new file mode 100644
index 0000000..c63b951
--- /dev/null
+++ b/lens-cube/src/test/resources/schema/facts/testfact7_base.xml
@@ -0,0 +1,67 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+
+  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.
+
+-->
+<x_fact_table name="testfact7_base" cube_name="virtualcube" weight="5.0" xmlns="uri:lens:cube:0.1">
+  <columns>
+    <column name="msr3" _type="double" comment="msr3"/>
+    <column name="cityid" _type="int" comment="city id"/>
+    <column name="stateid" _type="int" comment="state id"/>
+  </columns>
+  <properties>
+    <property name="cube.fact.testfact.c5.updateperiods"
+              value="MONTHLY,HOURLY,YEARLY,CONTINUOUS,QUARTERLY,MINUTELY,DAILY"/>
+    <property name="cube.fact.testfact.cubename" value="testCube"/>
+    <property name="cube.fact.relative.start.time" value="now.year - 90 days"/>
+    <property name="cube.fact.testfact.c4.updateperiods"
+              value="MONTHLY,HOURLY,YEARLY,CONTINUOUS,QUARTERLY,MINUTELY,DAILY"/>
+    <property name="cube.table.testfact.weight" value="5.0"/>
+    <property name="cube.fact.testfact.c3.updateperiods"
+              value="MONTHLY,HOURLY,YEARLY,CONTINUOUS,QUARTERLY,MINUTELY,DAILY"/>
+    <property name="cube.fact.testfact.c2.updateperiods"
+              value="MONTHLY,HOURLY,YEARLY,CONTINUOUS,QUARTERLY,MINUTELY,DAILY"/>
+    <property name="cube.fact.testfact.storages" value="C3,C4,C5,C0,C1,C2"/>
+    <property name="cube.fact.testfact.c0.updateperiods"
+              value="MONTHLY,HOURLY,YEARLY,CONTINUOUS,QUARTERLY,MINUTELY,DAILY"/>
+    <property name="cube.fact.testfact.c1.updateperiods"
+              value="MONTHLY,HOURLY,YEARLY,CONTINUOUS,QUARTERLY,MINUTELY,DAILY"/>
+  </properties>
+  <storage_tables>
+    <storage_table>
+      <update_periods>
+        <update_period>HOURLY</update_period>
+        <update_period>DAILY</update_period>
+      </update_periods>
+      <storage_name>C1</storage_name>
+      <table_desc external="false">
+        <part_cols>
+          <column name="dt" _type="string" comment="date partition"/>
+        </part_cols>
+        <table_parameters>
+          <property name="cube.storagetable.time.partcols" value="dt"/>
+        </table_parameters>
+        <serde_parameters>
+          <property name="serialization.format" value="1"/>
+        </serde_parameters>
+        <time_part_cols>dt</time_part_cols>
+      </table_desc>
+    </storage_table>
+  </storage_tables>
+</x_fact_table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lens/blob/fbad3507/lens-cube/src/test/resources/schema/facts/testfact8_base.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/testfact8_base.xml b/lens-cube/src/test/resources/schema/facts/testfact8_base.xml
new file mode 100644
index 0000000..5450b59
--- /dev/null
+++ b/lens-cube/src/test/resources/schema/facts/testfact8_base.xml
@@ -0,0 +1,58 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+
+  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.
+
+-->
+<x_fact_table name="testfact8_base" cube_name="virtualcube" weight="5.0" xmlns="uri:lens:cube:0.1">
+  <columns>
+
+    <column name="msr2" _type="float" comment="second measure"/>
+    <column name="cityid" _type="int" comment="city id"/>
+    <column name="stateid" _type="int" comment="state id"/>
+  </columns>
+  <properties>
+    <property name="cube.fact.testfact.cubename" value="testCube"/>
+    <property name="cube.fact.relative.start.time" value="now.year - 90 days"/>
+    <property name="cube.table.testfact.weight" value="5.0"/>
+    <property name="cube.fact.testfact.storages" value="C1"/>
+    <property name="cube.fact.testfact.c1.updateperiods"
+              value="MONTHLY,HOURLY,YEARLY,CONTINUOUS,QUARTERLY,MINUTELY,DAILY"/>
+  </properties>
+  <storage_tables>
+    <storage_table>
+      <update_periods>
+        <update_period>HOURLY</update_period>
+        <update_period>DAILY</update_period>
+      </update_periods>
+      <storage_name>C1</storage_name>
+      <table_desc external="false">
+        <part_cols>
+          <column name="dt" _type="string" comment="date partition"/>
+        </part_cols>
+        <table_parameters>
+          <property name="cube.storagetable.time.partcols" value="dt"/>
+        </table_parameters>
+        <serde_parameters>
+          <property name="serialization.format" value="1"/>
+        </serde_parameters>
+        <time_part_cols>dt</time_part_cols>
+      </table_desc>
+    </storage_table>
+  </storage_tables>
+</x_fact_table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lens/blob/fbad3507/lens-cube/src/test/resources/schema/facts/testfact9_base.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/testfact9_base.xml b/lens-cube/src/test/resources/schema/facts/testfact9_base.xml
new file mode 100644
index 0000000..56e5746
--- /dev/null
+++ b/lens-cube/src/test/resources/schema/facts/testfact9_base.xml
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+
+  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.
+
+-->
+<x_fact_table name="testfact9_base" cube_name="testcube2" weight="5.0" xmlns="uri:lens:cube:0.1">
+  <columns>
+
+    <column name="msr1" _type="float" comment="second measure"/>
+  </columns>
+  <properties>
+    <property name="cube.fact.testfact.cubename" value="testCube"/>
+    <property name="cube.fact.relative.start.time" value="now.year - 90 days"/>
+    <property name="cube.table.testfact.weight" value="5.0"/>
+    <property name="cube.fact.testfact.storages" value="C1"/>
+    <property name="cube.fact.testfact.c1.updateperiods"
+              value="MONTHLY,HOURLY,YEARLY,CONTINUOUS,QUARTERLY,MINUTELY,DAILY"/>
+  </properties>
+  <storage_tables>
+    <storage_table>
+      <update_periods>
+        <update_period>HOURLY</update_period>
+        <update_period>DAILY</update_period>
+        <update_period>MONTHLY</update_period>
+      </update_periods>
+      <storage_name>C1</storage_name>
+      <table_desc external="false">
+        <part_cols>
+          <column name="dt" _type="string" comment="date partition"/>
+        </part_cols>
+        <table_parameters>
+          <property name="cube.storagetable.time.partcols" value="dt"/>
+        </table_parameters>
+        <serde_parameters>
+          <property name="serialization.format" value="1"/>
+        </serde_parameters>
+        <time_part_cols>dt</time_part_cols>
+      </table_desc>
+    </storage_table>
+  </storage_tables>
+</x_fact_table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lens/blob/fbad3507/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml b/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml
index 860965f..13f6773 100644
--- a/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml
+++ b/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml
@@ -19,7 +19,7 @@
   under the License.
 
 -->
-<x_virtual_fact_table source_fact_name="summary1" cube_name="virtualcube" name="virtualfact1" xmlns="uri:lens:cube:0.1"
+<x_virtual_fact_table source_fact_name="testfact9_base" cube_name="virtualcube" name="virtualfact1" xmlns="uri:lens:cube:0.1"
                       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="uri:lens:cube:0.1 cube-0.1.xsd ">
   <properties>
     <property name="cube.fact.query.where.filter" value=" dim1 = 10 "/>


[05/50] lens git commit: LENS-1438: Testcase failure after LENS-1432

Posted by ra...@apache.org.
LENS-1438: Testcase failure after LENS-1432


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

Branch: refs/heads/master
Commit: 42d60d2623fae53609be93c5c71ece4e2adc267e
Parents: 28aa4a1
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Fri Jun 16 14:52:57 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Mon Jun 19 09:04:52 2017 +0800

----------------------------------------------------------------------
 .../lens/server/rewrite/TestRewriting.java      | 31 ++++++++++----------
 1 file changed, 15 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/42d60d26/lens-server/src/test/java/org/apache/lens/server/rewrite/TestRewriting.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/rewrite/TestRewriting.java b/lens-server/src/test/java/org/apache/lens/server/rewrite/TestRewriting.java
index 83dd1f7..304d21e 100644
--- a/lens-server/src/test/java/org/apache/lens/server/rewrite/TestRewriting.java
+++ b/lens-server/src/test/java/org/apache/lens/server/rewrite/TestRewriting.java
@@ -25,6 +25,7 @@ import org.apache.lens.cube.metadata.CubeMetastoreClient;
 import org.apache.lens.cube.parse.CubeQueryContext;
 import org.apache.lens.cube.parse.CubeQueryRewriter;
 import org.apache.lens.cube.parse.HQLParser;
+import org.apache.lens.cube.parse.QueryWriterContext;
 import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.driver.LensDriver;
 import org.apache.lens.server.api.driver.MockDriver;
@@ -55,6 +56,7 @@ import org.testng.annotations.ObjectFactory;
 import org.testng.annotations.Test;
 
 import com.codahale.metrics.MetricRegistry;
+import com.google.common.collect.Maps;
 
 /**
  * The Class TestRewriting.
@@ -96,24 +98,18 @@ public class TestRewriting {
 
   private CubeQueryRewriter getMockedRewriter() throws ParseException, LensException, HiveException {
     CubeQueryRewriter mockwriter = Mockito.mock(CubeQueryRewriter.class);
-    Mockito.when(mockwriter.rewrite(Matchers.any(String.class))).thenAnswer(new Answer<CubeQueryContext>() {
-      @Override
-      public CubeQueryContext answer(InvocationOnMock invocation) throws Throwable {
-        Object[] args = invocation.getArguments();
-        // return query for first NUM_SUCCESS calls and fail later
-        if (++i <= NUM_SUCCESS) {
-          return getMockedCubeContext((String) args[0]);
-        } else {
-          throw new RuntimeException("Mock fail");
-        }
+    Mockito.when(mockwriter.rewrite(Matchers.any(String.class))).thenAnswer(invocation -> {
+      Object[] args = invocation.getArguments();
+      // return query for first NUM_SUCCESS calls and fail later
+      if (++i <= NUM_SUCCESS) {
+        return getMockedCubeContext((String) args[0]);
+      } else {
+        throw new RuntimeException("Mock fail");
       }
     });
-    Mockito.when(mockwriter.rewrite(Matchers.any(ASTNode.class))).thenAnswer(new Answer<CubeQueryContext>() {
-      @Override
-      public CubeQueryContext answer(InvocationOnMock invocation) throws Throwable {
-        Object[] args = invocation.getArguments();
-        return getMockedCubeContext((ASTNode) args[0]);
-      }
+    Mockito.when(mockwriter.rewrite(Matchers.any(ASTNode.class))).thenAnswer(invocation -> {
+      Object[] args = invocation.getArguments();
+      return getMockedCubeContext((ASTNode) args[0]);
     });
     Mockito.doCallRealMethod().when(mockwriter).clear();
     return mockwriter;
@@ -130,6 +126,9 @@ public class TestRewriting {
   private CubeQueryContext getMockedCubeContext(String query)
     throws ParseException, LensException {
     CubeQueryContext context = Mockito.mock(CubeQueryContext.class);
+    QueryWriterContext mockQueryWriterContext = Mockito.mock(QueryWriterContext.class);
+    Mockito.when(mockQueryWriterContext.getDimsToQuery()).thenReturn(Maps.newHashMap());
+    Mockito.when(context.getQueryWriterContext()).thenReturn(mockQueryWriterContext);
     Mockito.when(context.toHQL()).thenReturn(query.substring(4));
     Mockito.when(context.toAST(Matchers.any(Context.class)))
       .thenReturn(HQLParser.parseHQL(query.toLowerCase().replaceFirst("^cube", ""), hconf));


[37/50] lens git commit: LENS-1481: Add License headers to .js files

Posted by ra...@apache.org.
LENS-1481: Add License headers to .js files


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

Branch: refs/heads/master
Commit: c566bf61e009e6cdc4a2b71ed699680d7c20c323
Parents: 2772efb
Author: rajub <ra...@lazada.com>
Authored: Mon Nov 6 13:42:11 2017 +0800
Committer: rajub <ra...@lazada.com>
Committed: Mon Nov 6 13:42:11 2017 +0800

----------------------------------------------------------------------
 lens-ui/app/actions/SessionAction.js           | 18 ++++++++++++++++++
 lens-ui/app/adapters/SessionAdapter.js         | 18 ++++++++++++++++++
 lens-ui/app/components/SessionListComponent.js | 18 ++++++++++++++++++
 lens-ui/app/constants/SessionConstants.js      | 18 ++++++++++++++++++
 lens-ui/app/stores/SessionStore.js             | 18 ++++++++++++++++++
 5 files changed, 90 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/c566bf61/lens-ui/app/actions/SessionAction.js
----------------------------------------------------------------------
diff --git a/lens-ui/app/actions/SessionAction.js b/lens-ui/app/actions/SessionAction.js
index c3d78f9..17eccd5 100644
--- a/lens-ui/app/actions/SessionAction.js
+++ b/lens-ui/app/actions/SessionAction.js
@@ -1,3 +1,21 @@
+/**
+ * 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.
+ */
 import AppDispatcher from "../dispatcher/AppDispatcher";
 import SessionConstants from "../constants/SessionConstants";
 import SessionAdapter from "../adapters/SessionAdapter";

http://git-wip-us.apache.org/repos/asf/lens/blob/c566bf61/lens-ui/app/adapters/SessionAdapter.js
----------------------------------------------------------------------
diff --git a/lens-ui/app/adapters/SessionAdapter.js b/lens-ui/app/adapters/SessionAdapter.js
index 8e4fe74..105ad8b 100644
--- a/lens-ui/app/adapters/SessionAdapter.js
+++ b/lens-ui/app/adapters/SessionAdapter.js
@@ -1,3 +1,21 @@
+/**
+ * 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.
+ */
 import BaseAdapter from "./BaseAdapter";
 import Config from 'config.json';
 

http://git-wip-us.apache.org/repos/asf/lens/blob/c566bf61/lens-ui/app/components/SessionListComponent.js
----------------------------------------------------------------------
diff --git a/lens-ui/app/components/SessionListComponent.js b/lens-ui/app/components/SessionListComponent.js
index a71281c..8e88174 100644
--- a/lens-ui/app/components/SessionListComponent.js
+++ b/lens-ui/app/components/SessionListComponent.js
@@ -1,3 +1,21 @@
+/**
+ * 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.
+ */
 import React from "react";
 import Session from "../stores/SessionStore";
 import SessionAction from "../actions/SessionAction";

http://git-wip-us.apache.org/repos/asf/lens/blob/c566bf61/lens-ui/app/constants/SessionConstants.js
----------------------------------------------------------------------
diff --git a/lens-ui/app/constants/SessionConstants.js b/lens-ui/app/constants/SessionConstants.js
index 3c31e5d..4088eed 100644
--- a/lens-ui/app/constants/SessionConstants.js
+++ b/lens-ui/app/constants/SessionConstants.js
@@ -1,3 +1,21 @@
+/**
+ * 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.
+ */
 import KeyMirror from 'keymirror';
 
 const SessionConstants = KeyMirror({

http://git-wip-us.apache.org/repos/asf/lens/blob/c566bf61/lens-ui/app/stores/SessionStore.js
----------------------------------------------------------------------
diff --git a/lens-ui/app/stores/SessionStore.js b/lens-ui/app/stores/SessionStore.js
index 3840720..359da4f 100644
--- a/lens-ui/app/stores/SessionStore.js
+++ b/lens-ui/app/stores/SessionStore.js
@@ -1,3 +1,21 @@
+/**
+ * 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.
+ */
 import assign from "object-assign";
 import SessionConstants from "../constants/SessionConstants";
 import AppDispatcher from "../dispatcher/AppDispatcher";


[20/50] lens git commit: LENS-1455: Getting expression ast to generate default expression should consider all expression contexts

Posted by ra...@apache.org.
LENS-1455: Getting expression ast to generate default expression should consider all expression contexts


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

Branch: refs/heads/master
Commit: cdd7b09995b765783867812e5b981c27f4938371
Parents: 8b41202
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Thu Jul 20 16:11:15 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 20 16:13:07 2017 +0530

----------------------------------------------------------------------
 .../cube/parse/CandidateCoveringSetsResolver.java  |  3 +++
 .../apache/lens/cube/parse/ExpressionResolver.java | 17 ++++++++++++-----
 .../lens/cube/parse/SegmentationCandidate.java     |  5 +++++
 3 files changed, 20 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/cdd7b099/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
index 4066cf7..de30703 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
@@ -255,6 +255,9 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
         i.remove();
       }
     }
+    // sorting will make sure storage candidates come before complex candidates.
+    // ensuring maximum columns get selected from simpler candidates.
+    ucSet.sort(Comparator.comparing(Candidate::getChildrenCount));
     // Sets that contain all measures or no measures are removed from iteration.
     // find other facts
     for (Iterator<Candidate> i = ucSet.iterator(); i.hasNext();) {

http://git-wip-us.apache.org/repos/asf/lens/blob/cdd7b099/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
index 66b043e..4680766 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hive.ql.parse.HiveParser;
 
 import org.antlr.runtime.CommonToken;
 
+import com.google.common.collect.Sets;
 import lombok.*;
 import lombok.extern.slf4j.Slf4j;
 
@@ -501,15 +502,21 @@ class ExpressionResolver implements ContextRewriter {
       }
     }
 
-    private ASTNode getExprAst(ExpressionContext ec) {
+    private ASTNode getExprAst(ExpressionContext expressionContext) {
       Set<StorageCandidate> scSet = CandidateUtil.getStorageCandidates(cubeql.getCandidates());
-      Set<String> storageTableNames = new HashSet<String>();
+      Set<String> storageTableNames = new HashSet<>();
+      Set<ExpressionContext> expressionContexts = Sets.newLinkedHashSet();
+      expressionContexts.add(expressionContext);
       for (StorageCandidate sc : scSet) {
         storageTableNames.add(sc.getStorageTable());
+        expressionContexts.add(sc.getCubeQueryContext().getExprCtx()
+          .getExpressionContext(expressionContext.getExprCol().getName(), expressionContext.getSrcAlias()));
       }
-      for (CandidateTable table : ec.evaluableExpressions.keySet()) {
-        if (storageTableNames.contains(table.getStorageTable())) {
-          return  MetastoreUtil.copyAST(ec.evaluableExpressions.get(table).iterator().next().finalAST);
+      for (ExpressionContext ec : expressionContexts) {
+        for (CandidateTable table : ec.evaluableExpressions.keySet()) {
+          if (storageTableNames.contains(table.getStorageTable())) {
+            return MetastoreUtil.copyAST(ec.evaluableExpressions.get(table).iterator().next().finalAST);
+          }
         }
       }
       return null;

http://git-wip-us.apache.org/repos/asf/lens/blob/cdd7b099/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
index 35638f3..055d6ef 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
@@ -273,6 +273,11 @@ public class SegmentationCandidate implements Candidate {
   }
 
   @Override
+  public int getChildrenCount() {
+    return segmentation.getSegments().size();
+  }
+
+  @Override
   public boolean isTimeRangeCoverable(TimeRange timeRange) throws LensException {
     return true;
   }


[14/50] lens git commit: LENS-1446: Cube latestdate api broken after virtual fact

Posted by ra...@apache.org.
LENS-1446: Cube latestdate api broken after virtual fact


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

Branch: refs/heads/master
Commit: dd3b1bd19d82fb64441d9982a5aa02b7f849e3a7
Parents: 54ab131
Author: Rajitha R <ra...@gmail.com>
Authored: Thu Jun 29 17:23:26 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:42:56 2017 +0530

----------------------------------------------------------------------
 .../apache/lens/cli/commands/LensSchemaCommands.java |  2 +-
 .../lens/cube/metadata/CubeMetastoreClient.java      | 15 ++++++++-------
 .../org/apache/lens/cube/parse/StorageCandidate.java |  8 ++++----
 3 files changed, 13 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/dd3b1bd1/lens-cli/src/main/java/org/apache/lens/cli/commands/LensSchemaCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/main/java/org/apache/lens/cli/commands/LensSchemaCommands.java b/lens-cli/src/main/java/org/apache/lens/cli/commands/LensSchemaCommands.java
index aca1cf9..cc184a8 100644
--- a/lens-cli/src/main/java/org/apache/lens/cli/commands/LensSchemaCommands.java
+++ b/lens-cli/src/main/java/org/apache/lens/cli/commands/LensSchemaCommands.java
@@ -113,7 +113,7 @@ public class LensSchemaCommands implements CommandMarker {
     CREATE_COMMAND_MAP.put(XFactTable.class, "create fact --path %s");
     UPDATE_COMMAND_MAP.put(XFactTable.class, "update fact --fact_name %s --path %s");
     CREATE_COMMAND_MAP.put(XVirtualFactTable.class, "create fact --path %s");
-    UPDATE_COMMAND_MAP.put(XVirtualFactTable.class, "update fact --name %s --path %s");
+    UPDATE_COMMAND_MAP.put(XVirtualFactTable.class, "update fact --fact_name %s --path %s");
     CREATE_COMMAND_MAP.put(XSegmentation.class, "create segmentation --path %s");
     UPDATE_COMMAND_MAP.put(XSegmentation.class, "update segmentation --name %s --path %s");
   }

http://git-wip-us.apache.org/repos/asf/lens/blob/dd3b1bd1/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
index 5c05e53..3952696 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
@@ -161,7 +161,8 @@ public class CubeMetastoreClient {
     for (FactTable fact : getAllFacts(cube)) {
       for (String storage : fact.getStorages()) {
         for (UpdatePeriod updatePeriod : fact.getUpdatePeriods().get(storage)) {
-          PartitionTimeline timeline = partitionTimelineCache.get(fact.getName(), storage, updatePeriod, partCol);
+          PartitionTimeline timeline = partitionTimelineCache.get(fact.getSourceFactName(), storage, updatePeriod,
+            partCol);
           if (timeline != null) {// this storage table is partitioned by partCol or not.
             Date latest = timeline.getLatestDate();
             if (latest != null && latest.after(max)) {
@@ -1078,8 +1079,8 @@ public class CubeMetastoreClient {
       List<Partition> partsAdded = new ArrayList<>();
       // first update in memory, then add to hive table's partitions. delete is reverse.
       partitionTimelineCache.updateForAddition(factOrDimTable, storageName, updatePeriod,
-              getTimePartSpecs(storagePartitionDescs, getStorageTableStartDate(storageTableName, factOrDimTable),
-                      getStorageTableEndDate(storageTableName, factOrDimTable)));
+              getTimePartSpecs(storagePartitionDescs, getStorageTableStartDate(storageTableName,
+                getFactTable(factOrDimTable)), getStorageTableEndDate(storageTableName, getFactTable(factOrDimTable))));
       // Adding partition in fact table.
       if (storagePartitionDescs.size() > 0) {
         partsAdded = getStorage(storageName).addPartitions(getClient(), factOrDimTable, updatePeriod,
@@ -1094,17 +1095,17 @@ public class CubeMetastoreClient {
     }
   }
 
-  public Date getStorageTableStartDate(String storageTable, String factTableName)
+  public Date getStorageTableStartDate(String storageTable, FactTable factTableName)
     throws LensException {
     List<Date> startDates = getStorageTimes(storageTable, MetastoreUtil.getStoragetableStartTimesKey());
-    startDates.add(getFactTable(factTableName).getStartTime());
+    startDates.add(factTableName.getStartTime());
     return Collections.max(startDates);
   }
 
-  public Date getStorageTableEndDate(String storageTable, String factTableName)
+  public Date getStorageTableEndDate(String storageTable, FactTable factTableName)
     throws LensException {
     List<Date> endDates = getStorageTimes(storageTable, MetastoreUtil.getStoragetableEndTimesKey());
-    endDates.add((getFactTable(factTableName)).getEndTime());
+    endDates.add(factTableName.getEndTime());
     return Collections.min(endDates);
   }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/dd3b1bd1/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
index c6ef6d2..2b1add3 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
@@ -252,8 +252,8 @@ public class StorageCandidate implements Candidate, CandidateTable {
     List<Date> startDates = new ArrayList<>();
     List<Date> endDates = new ArrayList<>();
     for (String storageTablePrefix : getValidStorageTableNames()) {
-      startDates.add(getCubeMetastoreClient().getStorageTableStartDate(storageTablePrefix, fact.getSourceFactName()));
-      endDates.add(getCubeMetastoreClient().getStorageTableEndDate(storageTablePrefix, fact.getSourceFactName()));
+      startDates.add(getCubeMetastoreClient().getStorageTableStartDate(storageTablePrefix, fact));
+      endDates.add(getCubeMetastoreClient().getStorageTableEndDate(storageTablePrefix, fact));
     }
     this.startTime = Collections.min(startDates);
     this.endTime = Collections.max(endDates);
@@ -891,7 +891,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
       return this.startTime;
     }
     return getCubeMetastoreClient().getStorageTableStartDate(
-      getCubeMetastoreClient().getStorageTableName(fact.getSourceFactName(), storageName, interval), fact.getName());
+      getCubeMetastoreClient().getStorageTableName(fact.getSourceFactName(), storageName, interval), fact);
   }
 
   private Date getStorageTableEndDate(UpdatePeriod interval) throws LensException {
@@ -900,7 +900,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
       return this.endTime;
     }
     return getCubeMetastoreClient().getStorageTableEndDate(
-      getCubeMetastoreClient().getStorageTableName(fact.getSourceFactName(), storageName, interval), fact.getName());
+      getCubeMetastoreClient().getStorageTableName(fact.getSourceFactName(), storageName, interval), fact);
   }
 
 


[24/50] lens git commit: LENS-1450 : Fix duplicate filters in query writing

Posted by ra...@apache.org.
LENS-1450 : Fix duplicate filters in query writing


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

Branch: refs/heads/master
Commit: 05c364e24820b8b6e10a226a523429ca35f9e4d4
Parents: 1f9f12f
Author: Rajitha R <ra...@gmail.com>
Authored: Mon Aug 7 17:29:26 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Mon Aug 21 13:00:48 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/lens/cube/parse/CubeQueryContext.java |  3 ++-
 .../java/org/apache/lens/cube/parse/TestCubeRewriter.java | 10 +++++++++-
 2 files changed, 11 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/05c364e2/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 b5330a7..8b9583a 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
@@ -1169,7 +1169,8 @@ public class CubeQueryContext extends TracksQueriedColumns implements QueryAST,
         }
       }
     } else if (node.getParent() == null
-        && node.getToken().getType() != HiveParser.KW_AND) {
+        && node.getToken().getType() != HiveParser.KW_AND
+      && node.getChild(0).getType() != HiveParser.KW_AND) {
       // if node is the only child
       allFilters.add(HQLParser.getString((ASTNode) node));
     }

http://git-wip-us.apache.org/repos/asf/lens/blob/05c364e2/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 1c75e6c..bc0fa2e 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
@@ -509,8 +509,16 @@ public class TestCubeRewriter extends TestQueryRewrite {
     conf.setBoolean(DISABLE_AUTO_JOINS, false);
     conf.setBoolean(REWRITE_DIM_FILTER_TO_FACT_FILTER, true);
 
+    // No filter
+    String hql = rewrite("select cityid , msr2 from testCube where " + TWO_DAYS_RANGE, conf);
+
+    String expectedQuery =  getExpectedQuery(TEST_CUBE_NAME, "select (testcube.cityid) as `cityid`, "
+        + "sum((testcube.msr2)) as `msr2` from ", null, "group by testcube.cityid",
+      getWhereForHourly2days(TEST_CUBE_NAME, "c3_testfact2_raw"));
+    compareQueries(expectedQuery, hql);
+
     // filter with =
-    String hql = rewrite(
+    hql = rewrite(
         "select cubecountry.name, msr2 from" + " testCube" + " where cubecountry.region = 'asia' and "
             + TWO_DAYS_RANGE, conf);
     String filterSubquery = "testcube.countryid in ( select id from TestQueryRewrite.c3_countrytable_partitioned "


[38/50] lens git commit: LENS-1425: Update 2.7 release documentation

Posted by ra...@apache.org.
LENS-1425: Update 2.7 release documentation


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

Branch: refs/heads/master
Commit: b423d5c4a57c3e18bdfac40153a22d103dc771b8
Parents: c566bf6
Author: rajub <ra...@lazada.com>
Authored: Tue Nov 28 13:39:23 2017 +0800
Committer: rajub <ra...@lazada.com>
Committed: Tue Nov 28 13:39:23 2017 +0800

----------------------------------------------------------------------
 README.md                                 |  4 ++--
 src/site/apt/releases/download.apt        |  2 +-
 src/site/apt/releases/release-history.apt | 10 ++++++++++
 3 files changed, 13 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/b423d5c4/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 169791d..732dcd9 100644
--- a/README.md
+++ b/README.md
@@ -5,7 +5,7 @@ Apache Lens is a unified Analytics Platform. Lens aims to cut the Data Analytics
 across multiple tiered data stores and optimal execution environment for the analytical query.
 
 Prerequisites :
-Apache Lens requires JDK(>=1.7) and Apache Maven(3.x) to be installed for the build.
+Apache Lens requires JDK(>=1.8) and Apache Maven(3.x) to be installed for the build.
 
 JAVA_HOME is required for running tests.
 
@@ -15,7 +15,7 @@ Confirm versions :
   # echo ${JAVA_HOME}
 
 Additionally MAVEN_OPTS can be configured as :
-  # export MAVEN_OPTS="-Xmx1024m -XX:MaxPermSize=256m"
+  # export MAVEN_OPTS="-Xmx1024m"
 
 Build the project :
   # mvn clean package

http://git-wip-us.apache.org/repos/asf/lens/blob/b423d5c4/src/site/apt/releases/download.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/releases/download.apt b/src/site/apt/releases/download.apt
index df4df1d..a190053 100644
--- a/src/site/apt/releases/download.apt
+++ b/src/site/apt/releases/download.apt
@@ -20,7 +20,7 @@
 Download
 
   The latest release of Apache Lens can be
-  {{{http://www.apache.org/dyn/closer.lua/lens/2.6.1}downloaded from the ASF}}.
+  {{{http://www.apache.org/dyn/closer.lua/lens/2.7.0}downloaded from the ASF}}.
 
   Older releases may be found {{{http://archive.apache.org/dist/lens}in the archives}}.
   Releases prior to graduation may be found {{{http://archive.apache.org/dist/incubator/lens}in the incubator archives}}.

http://git-wip-us.apache.org/repos/asf/lens/blob/b423d5c4/src/site/apt/releases/release-history.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/releases/release-history.apt b/src/site/apt/releases/release-history.apt
index 7b9723b..2f687c7 100644
--- a/src/site/apt/releases/release-history.apt
+++ b/src/site/apt/releases/release-history.apt
@@ -23,6 +23,8 @@ All Apache Lens releases
 *--+--+---+--+--+--+--+
 |Release version|Major features|Release documentation|Release Notes|Download|Incompatibilities|More Info |
 *--+--+---+--+--+--+--+
+|2.7.0| Cube segmentation, Unioning data across facts, Support for source data completion checking, Added Retry feature for transient errors on query execution, Added an ability to analyze active sessions, Moving Lens to java8, Support for PreparedStatement in Lens JDBC client, Notifications on query completpletion, Limiting the max number of jobs scheduled per user, Support for creating the schema from CLI, Code cleanup and refactor, Doc/javadoc improvements, Bug fixes | {{{../versions/2.7.0/index.html} 2.7.x docs}} | {{{https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315923&version=12336851} 2.7 release notes}} | {{{http://archive.apache.org/dist/lens/2.7.0/} Download}} | {{{#a2.7.x_from_2.6.x} 2.7.x-incompatibilities}} | - |
+*--+--+---+--+--+--+--+
 |2.6.1| GA release, Apache Hive 2 support, Query Scheduler, Lens UI enhancements, Python client, Streaming results, New regression module, Weighted Driver selector, Limit number of open sessions, Exponential back off for hive status updates, Single execution for duplicate queries, Auto kill queries on timeout, Convert dimension filter to fact filters for performance improvement, Code cleanup and refactor, Doc/javadoc improvements, Bug fixes| {{{../versions/2.6.1/index.html} 2.6.x docs}} |{{{https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12333846&styleName=&projectId=12315923} 2.6 release notes}} |{{{http://archive.apache.org/dist/lens/2.6.1/} Download}} |{{{#a2.6.x_from_2.5.x} 2.6.x-incompatibilities}} | - |
 *--+--+---+--+--+--+--+
 |2.5.0-beta| Support for multiple instances of a driver, REST API enhancements, Derived cube validation, Per-queue driver max launched queries constraint, New error codes for Hive and JDBC Driver, Lens CLI improvements, Query result retention policy, UI Enhancements, Code cleanup and refactor, Doc/javadoc improvements, Bug fixes| {{{../versions/2.5.0-beta/index.html} 2.5.x docs}} |{{{https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12333326&projectId=12315923} 2.5 release notes}} |{{{http://archive.apache.org/dist/lens/2.5-beta/} Download}} |{{{#a2.5.x_from_2.4.x} 2.5.x-incompatibilities}} | - |
@@ -46,6 +48,14 @@ All Apache Lens releases
 
 * Incompatibilities
 
+** 2.7.x from 2.6.x
+
+  * LENS-1387: Moved Lens to Java8
+
+  * LENS-1317: Finished_Queries store stores the LensConf
+
+  * LENS-1304: Storing Driver query in finished_queries tables for better analytics purposes
+
 ** 2.6.x from 2.5.x
 
   * LENS-919: For older deployments the schema for "finished_queries" table needs to be modified to include "priority"


[16/50] lens git commit: LENS-1449 : lens.query.result.parent.dir to be set at driver level.

Posted by ra...@apache.org.
LENS-1449 : lens.query.result.parent.dir to be set at driver level.


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

Branch: refs/heads/master
Commit: 41051ea47f164f004f5fccaa7ea91765007811fc
Parents: 503a46d
Author: Raghavendra Singh <ra...@apache.org>
Authored: Thu Jul 6 14:03:59 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:42:59 2017 +0530

----------------------------------------------------------------------
 .../api/query/DriverSelectorQueryContext.java   |  4 +++-
 .../lens/server/api/query/QueryContext.java     |  5 +++++
 .../lens/server/query/TestQueryService.java     |  6 ++++--
 .../lens/server/query/TestResultFormatting.java | 20 +++++++++++++++++++-
 .../drivers/hive/hive1/hivedriver-site.xml      |  5 +++++
 .../drivers/hive/hive2/hivedriver-site.xml      |  6 ++++++
 .../drivers/jdbc/jdbc1/jdbcdriver-site.xml      |  5 +++++
 lens-server/src/test/resources/lens-site.xml    |  5 -----
 8 files changed, 47 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/41051ea4/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
index 8e431d1..41e533e 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
@@ -155,7 +155,9 @@ public class DriverSelectorQueryContext {
   private Configuration mergeConf(LensDriver driver, Configuration queryConf) {
     Configuration conf = new Configuration(driver.getConf());
     for (Map.Entry<String, String> entry : queryConf) {
-      conf.set(entry.getKey(), entry.getValue());
+      if (!conf.getFinalParameters().contains(entry.getKey())) {
+        conf.set(entry.getKey(), entry.getValue());
+      }
     }
     conf.setClassLoader(queryConf.getClassLoader());
     return conf;

http://git-wip-us.apache.org/repos/asf/lens/blob/41051ea4/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 9923589..ccdef87 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
@@ -328,6 +328,11 @@ public class QueryContext extends AbstractQueryContext implements FailureContext
   }
 
   public String getResultSetParentDir() {
+    if (getSelectedDriver() != null && getSelectedDriverConf().get(LensConfConstants.RESULT_SET_PARENT_DIR) != null) {
+      log.info("Fetching Parent Dir from driver conf:- "
+              + getSelectedDriverConf().get(LensConfConstants.RESULT_SET_PARENT_DIR));
+      return getSelectedDriverConf().get(LensConfConstants.RESULT_SET_PARENT_DIR);
+    }
     return conf.get(LensConfConstants.RESULT_SET_PARENT_DIR, LensConfConstants.RESULT_SET_PARENT_DIR_DEFAULT);
   }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/41051ea4/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 1149696..b7cdb88 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
@@ -848,9 +848,11 @@ public class TestQueryService extends LensJerseyTest {
     LensResultSet rs = queryService.getResultset(handle3);
     //check persisted result path
     String expectedPath =
-        ctx3.getConf().get(LensConfConstants.RESULT_SET_PARENT_DIR) + "/" + handle3.getHandleIdString()
+        ctx3.getSelectedDriverConf().get(LensConfConstants.RESULT_SET_PARENT_DIR) + "/" + handle3.getHandleIdString()
             + ctx3.getConf().get(LensConfConstants.QUERY_OUTPUT_FILE_EXTN);
-    assertTrue(((PersistentResultSet) rs).getOutputPath().endsWith(expectedPath));
+    assertTrue(((PersistentResultSet) rs).getOutputPath().endsWith(expectedPath)
+            , "Result Path " + ((PersistentResultSet) rs).getOutputPath()
+            + " does not contain expected path: " + expectedPath);
 
     validateHttpEndPoint(target(), null, handle3, null);
   }

http://git-wip-us.apache.org/repos/asf/lens/blob/41051ea4/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 4f8b8ff..76d3720 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
@@ -157,6 +157,22 @@ public class TestResultFormatting extends LensJerseyTest {
   }
 
   /**
+   * Test result formatter hdfs persistent result location override from driver conf.
+   *
+   * @throws InterruptedException the interrupted exception
+   * @throws IOException          Signals that an I/O exception has occurred.
+   */
+  @Test
+  public void testResultDirOverrideFromDriverConf() throws InterruptedException, IOException {
+    LensConf conf = new LensConf();
+    conf.addProperty(LensConfConstants.QUERY_PERSISTENT_RESULT_INDRIVER, "true");
+    LensQuery lensQuery = testResultFormatter(conf, QueryStatus.Status.SUCCESSFUL, false,
+            null, MediaType.APPLICATION_XML_TYPE);
+    assertEquals(lensQuery.getResultSetPath(), "file:"+System.getProperty("user.dir")+"/target/hive-lens-results/"
+            +lensQuery.getQueryHandle().getHandleIdString()+".csv");
+  }
+
+  /**
    * Test persistent result with max size.
    *
    * @throws InterruptedException the interrupted exception
@@ -196,7 +212,7 @@ public class TestResultFormatting extends LensJerseyTest {
    * @throws InterruptedException the interrupted exception
    * @throws IOException          Signals that an I/O exception has occurred.
    */
-  private void testResultFormatter(LensConf conf, Status status, boolean isDir, String reDirectUrl, MediaType mt)
+  private LensQuery testResultFormatter(LensConf conf, Status status, boolean isDir, String reDirectUrl, MediaType mt)
     throws InterruptedException, IOException {
     // test post execute op
     final WebTarget target = target().path("queryapi/queries");
@@ -266,6 +282,8 @@ public class TestResultFormatting extends LensJerseyTest {
         || ctx.getStatus().getStatusMessage().equals(ResultFormatter.ERROR_MESSAGE));
       assertEquals(ctx.getStatus().getErrorMessage(), "Class NonexistentSerde.class not found");
     }
+
+    return ctx;
   }
 
   @AfterTest

http://git-wip-us.apache.org/repos/asf/lens/blob/41051ea4/lens-server/src/test/resources/drivers/hive/hive1/hivedriver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/drivers/hive/hive1/hivedriver-site.xml b/lens-server/src/test/resources/drivers/hive/hive1/hivedriver-site.xml
index 1f0ff43..83143e7 100644
--- a/lens-server/src/test/resources/drivers/hive/hive1/hivedriver-site.xml
+++ b/lens-server/src/test/resources/drivers/hive/hive1/hivedriver-site.xml
@@ -82,4 +82,9 @@
     </description>
   </property>
 
+    <property>
+        <name>lens.query.result.parent.dir</name>
+        <value>target/hive-lens-results</value>
+        <final>true</final>
+    </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/41051ea4/lens-server/src/test/resources/drivers/hive/hive2/hivedriver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/drivers/hive/hive2/hivedriver-site.xml b/lens-server/src/test/resources/drivers/hive/hive2/hivedriver-site.xml
index eb902e0..4170e9b 100644
--- a/lens-server/src/test/resources/drivers/hive/hive2/hivedriver-site.xml
+++ b/lens-server/src/test/resources/drivers/hive/hive2/hivedriver-site.xml
@@ -82,4 +82,10 @@
     </description>
   </property>
 
+    <property>
+        <name>lens.query.result.parent.dir</name>
+        <value>target/hive-lens-results</value>
+        <final>true</final>
+    </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/41051ea4/lens-server/src/test/resources/drivers/jdbc/jdbc1/jdbcdriver-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/drivers/jdbc/jdbc1/jdbcdriver-site.xml b/lens-server/src/test/resources/drivers/jdbc/jdbc1/jdbcdriver-site.xml
index 5b8b43f..ad068f8 100644
--- a/lens-server/src/test/resources/drivers/jdbc/jdbc1/jdbcdriver-site.xml
+++ b/lens-server/src/test/resources/drivers/jdbc/jdbc1/jdbcdriver-site.xml
@@ -55,4 +55,9 @@
     <name>lens.driver.test.drivername</name>
     <value>jdbc/jdbc1</value>
   </property>
+    <property>
+        <name>lens.query.result.parent.dir</name>
+        <value>target/jdbc-lens-results</value>
+        <final>true</final>
+    </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/41051ea4/lens-server/src/test/resources/lens-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/lens-site.xml b/lens-server/src/test/resources/lens-site.xml
index 0060fa7..7e5f522 100644
--- a/lens-server/src/test/resources/lens-site.xml
+++ b/lens-server/src/test/resources/lens-site.xml
@@ -49,11 +49,6 @@
     <name>lens.server.persist.location</name>
     <value>target/persist-dir</value>
   </property>
-
-  <property>
-    <name>lens.query.result.parent.dir</name>
-    <value>target/lens-results</value>
-  </property>
   
   <property>
     <name>hive.server2.authentication</name>


[30/50] lens git commit: LENS-1473: Cubevirtualfact table relative start time has incorrect config field name

Posted by ra...@apache.org.
LENS-1473: Cubevirtualfact table relative start time has incorrect config field name


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

Branch: refs/heads/master
Commit: 717214cf9f17337fa6769f03afb3391b39b014ee
Parents: 358efef
Author: Rajitha R <ra...@gmail.com>
Authored: Wed Sep 6 18:33:38 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Thu Oct 5 11:12:08 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/lens/cube/metadata/CubeVirtualFactTable.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/717214cf/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeVirtualFactTable.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeVirtualFactTable.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeVirtualFactTable.java
index ce2a1a6..8b55985 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeVirtualFactTable.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeVirtualFactTable.java
@@ -147,7 +147,7 @@ public class CubeVirtualFactTable extends AbstractCubeTable implements FactTable
   }
 
   public Date getRelativeStartTime() {
-    String relativeStartTime = this.getProperties().get(MetastoreConstants.FACT_ABSOLUTE_START_TIME);
+    String relativeStartTime = this.getProperties().get(MetastoreConstants.FACT_RELATIVE_START_TIME);
     Date relativeDate = null;
     if (StringUtils.isNotBlank(relativeStartTime)) {
       relativeDate = MetastoreUtil.getDateFromProperty(relativeStartTime, true, true);


[40/50] lens git commit: LENS-1491: Disable testEstimateRejectionException and testSearchStoreJob tests temporarily

Posted by ra...@apache.org.
LENS-1491: Disable testEstimateRejectionException and testSearchStoreJob tests temporarily


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

Branch: refs/heads/master
Commit: 7fa91dd4ce978e93f29dbc64a28a295ce0559603
Parents: 0f08b69
Author: rajub <ra...@lazada.com>
Authored: Mon Dec 4 08:37:31 2017 +0800
Committer: rajub <ra...@lazada.com>
Committed: Mon Dec 4 08:37:31 2017 +0800

----------------------------------------------------------------------
 .../test/java/org/apache/lens/server/query/TestQueryService.java   | 2 +-
 .../java/org/apache/lens/server/scheduler/SchedulerDAOTest.java    | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/7fa91dd4/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 b7cdb88..3752132 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
@@ -2034,7 +2034,7 @@ public class TestQueryService extends LensJerseyTest {
       + (lensQuery.getFinishTime() - lensQuery.getLaunchTime()));
   }
 
-  @Test(dataProvider = "mediaTypeData")
+  //@Test(dataProvider = "mediaTypeData")
   public void testEstimateRejectionException(MediaType mt) throws Exception {
     class EstimateRunnable implements Runnable {
       boolean failed = false;

http://git-wip-us.apache.org/repos/asf/lens/blob/7fa91dd4/lens-server/src/test/java/org/apache/lens/server/scheduler/SchedulerDAOTest.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/scheduler/SchedulerDAOTest.java b/lens-server/src/test/java/org/apache/lens/server/scheduler/SchedulerDAOTest.java
index 4754c1e..d878cb1 100644
--- a/lens-server/src/test/java/org/apache/lens/server/scheduler/SchedulerDAOTest.java
+++ b/lens-server/src/test/java/org/apache/lens/server/scheduler/SchedulerDAOTest.java
@@ -192,7 +192,7 @@ public class SchedulerDAOTest {
     schedulerDAO.storeJob(info);
     // There should be 3 jobs till now.
     Assert.assertEquals(schedulerDAO.getJobs("lens", null, null, SchedulerJobState.values()).size(), 3);
-    Assert.assertEquals(schedulerDAO.getJobs("lens", 1L, System.currentTimeMillis(), SchedulerJobState.NEW).size(), 2);
+    //Assert.assertEquals(schedulerDAO.getJobs("lens", 1L, System.currentTimeMillis(), SchedulerJobState.NEW).size(), 2);
     Assert.assertEquals(schedulerDAO.getJobs("Alice", null, null, SchedulerJobState.NEW).size(), 0);
   }
 }


[29/50] lens git commit: LENS-1463 : Valid Columns addition in Virtual Fact

Posted by ra...@apache.org.
LENS-1463 : Valid Columns addition in Virtual Fact


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

Branch: refs/heads/master
Commit: 358efef9a3214f2c57655f2ea227b646971ae005
Parents: ff4f372
Author: Rajitha R <ra...@gmail.com>
Authored: Mon Aug 28 16:31:33 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Thu Oct 5 11:11:42 2017 +0800

----------------------------------------------------------------------
 .../lens/cube/metadata/CubeVirtualFactTable.java       |  9 ++++++++-
 .../apache/lens/cube/parse/TestVirtualFactQueries.java | 13 +++++++++++++
 .../src/test/resources/schema/cubes/base/testcube2.xml |  2 ++
 .../src/test/resources/schema/facts/testfact9_base.xml |  3 ++-
 .../resources/schema/facts/virtual/virtualfact.xml     |  2 ++
 5 files changed, 27 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/358efef9/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeVirtualFactTable.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeVirtualFactTable.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeVirtualFactTable.java
index 1fc74b0..ce2a1a6 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeVirtualFactTable.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeVirtualFactTable.java
@@ -92,7 +92,14 @@ public class CubeVirtualFactTable extends AbstractCubeTable implements FactTable
 
   @Override
   public Set<String> getValidColumns() {
-    return this.sourceCubeFactTable.getValidColumns();
+    String validColsStr =
+      MetastoreUtil.getNamedStringValue(this.getProperties(), MetastoreUtil.getValidColumnsKey(getName()));
+    if (validColsStr == null) {
+      return this.sourceCubeFactTable.getValidColumns();
+    } else {
+      return new HashSet<>(Arrays.asList(StringUtils.split(validColsStr.toLowerCase(),
+        ',')));
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lens/blob/358efef9/lens-cube/src/test/java/org/apache/lens/cube/parse/TestVirtualFactQueries.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestVirtualFactQueries.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestVirtualFactQueries.java
index 4bcb376..94e836c 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestVirtualFactQueries.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestVirtualFactQueries.java
@@ -25,8 +25,11 @@ import static org.apache.lens.cube.parse.CubeQueryConfUtil.DISABLE_AGGREGATE_RES
 import static org.apache.lens.cube.parse.CubeTestSetup.*;
 
 import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.fail;
 
+import org.apache.lens.cube.error.LensCubeErrorCode;
 import org.apache.lens.server.api.LensServerAPITestUtil;
+import org.apache.lens.server.api.error.LensException;
 
 import org.apache.hadoop.conf.Configuration;
 
@@ -119,5 +122,15 @@ public class TestVirtualFactQueries extends TestQueryRewrite {
 
   }
 
+  @Test
+  public void testVirtualFactValidColumns() throws Exception {
+
+    try {
+      rewriteCtx("select SUM(msr4) from virtualCube where " + TWO_DAYS_RANGE, getConfWithStorages("C1"));
+      fail("Rewrite should not succeed here");
+    } catch (LensException exc) {
+      assertEquals(exc.getErrorCode(), LensCubeErrorCode.COLUMN_NOT_FOUND.getLensErrorInfo().getErrorCode());
+    }
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/358efef9/lens-cube/src/test/resources/schema/cubes/base/testcube2.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/cubes/base/testcube2.xml b/lens-cube/src/test/resources/schema/cubes/base/testcube2.xml
index 9740e00..237e85a 100644
--- a/lens-cube/src/test/resources/schema/cubes/base/testcube2.xml
+++ b/lens-cube/src/test/resources/schema/cubes/base/testcube2.xml
@@ -27,5 +27,7 @@
   <measures>
     <measure _type="FLOAT" default_aggr="SUM" unit="RS" name="msr1" display_string="Measure1"
              description="first measure"/>
+    <measure _type="FLOAT" default_aggr="SUM" unit="RS" name="msr4" display_string="Measure4"
+             description="fourth measure"/>
   </measures>
 </x_base_cube>

http://git-wip-us.apache.org/repos/asf/lens/blob/358efef9/lens-cube/src/test/resources/schema/facts/testfact9_base.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/testfact9_base.xml b/lens-cube/src/test/resources/schema/facts/testfact9_base.xml
index 56e5746..9c94fe2 100644
--- a/lens-cube/src/test/resources/schema/facts/testfact9_base.xml
+++ b/lens-cube/src/test/resources/schema/facts/testfact9_base.xml
@@ -22,7 +22,8 @@
 <x_fact_table name="testfact9_base" cube_name="testcube2" weight="5.0" xmlns="uri:lens:cube:0.1">
   <columns>
 
-    <column name="msr1" _type="float" comment="second measure"/>
+    <column name="msr1" _type="float" comment="first measure"/>
+    <column name="msr4" _type="float" comment="fourth measure"/>
   </columns>
   <properties>
     <property name="cube.fact.testfact.cubename" value="testCube"/>

http://git-wip-us.apache.org/repos/asf/lens/blob/358efef9/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml b/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml
index 13f6773..a7daae5 100644
--- a/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml
+++ b/lens-cube/src/test/resources/schema/facts/virtual/virtualfact.xml
@@ -23,5 +23,7 @@
                       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="uri:lens:cube:0.1 cube-0.1.xsd ">
   <properties>
     <property name="cube.fact.query.where.filter" value=" dim1 = 10 "/>
+    <property name="cube.fact.virtualfact1.valid.columns"
+              value="msr1"/>
   </properties>
 </x_virtual_fact_table>


[28/50] lens git commit: LENS-1466 : Fix for 'Rewritten queries not resolving time_range_in clause'

Posted by ra...@apache.org.
LENS-1466 : Fix for 'Rewritten queries not resolving time_range_in clause'


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

Branch: refs/heads/master
Commit: ff4f3726a25db684ff7623fea68e954712dd925f
Parents: 10eef27
Author: Rajitha R <ra...@gmail.com>
Authored: Fri Sep 1 13:02:32 2017 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Fri Sep 1 15:23:45 2017 +0530

----------------------------------------------------------------------
 .../lens/cube/parse/StorageCandidate.java       |  2 +-
 .../apache/lens/cube/metadata/DateFactory.java  |  6 ++
 .../apache/lens/cube/parse/CubeTestSetup.java   | 47 +++++++++++--
 .../lens/cube/parse/TestCubeRewriter.java       | 15 ++++-
 .../resources/schema/cubes/base/testcube.xml    |  2 +
 .../test/resources/schema/facts/summary5.xml    | 70 ++++++++++++++++++++
 .../src/test/resources/schema/storages/c98.xml  | 26 ++++++++
 7 files changed, 159 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/ff4f3726/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
index 2b1add3..27835b3 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
@@ -533,9 +533,9 @@ public class StorageCandidate implements Candidate, CandidateTable {
           log.debug("Adding non existing partition {}", part);
           if (addNonExistingParts) {
             // Add non existing partitions for all cases of whether we populate all non existing or not.
-            this.participatingUpdatePeriods.add(maxInterval);
             missingPartitions.add(part);
             if (!failOnPartialData) {
+              this.participatingUpdatePeriods.add(maxInterval);
               partitions.add(part);
               part.getStorageTables().add(storageTable);
             }

http://git-wip-us.apache.org/repos/asf/lens/blob/ff4f3726/lens-cube/src/test/java/org/apache/lens/cube/metadata/DateFactory.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/metadata/DateFactory.java b/lens-cube/src/test/java/org/apache/lens/cube/metadata/DateFactory.java
index 855f54a..75f2abd 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/metadata/DateFactory.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/metadata/DateFactory.java
@@ -150,6 +150,7 @@ public class DateFactory {
 
   // Time Instances as Date Type
   public static final Date NOW;
+  public static final Date ONEDAY_BACK;
   public static final Date TWODAYS_BACK;
   public static final Date TWO_MONTHS_BACK;
   public static final Date THIS_MONTH_TRUNCATED;
@@ -168,6 +169,7 @@ public class DateFactory {
   public static final String TWO_DAYS_RANGE_TTD2;
   public static final String TWO_DAYS_RANGE_TTD2_BEFORE_4_DAYS;
   public static final String TWO_DAYS_RANGE_IT;
+  public static final String ONE_DAY_RANGE_IT;
   public static final String THIS_YEAR_RANGE;
   public static final String LAST_YEAR_RANGE;
   public static final String TWO_MONTHS_RANGE_UPTO_MONTH;
@@ -189,6 +191,9 @@ public class DateFactory {
     // Figure out if current hour is 0th hour
     zerothHour = getDateStringWithOffset(HOURLY, 0).endsWith("-00");
 
+    ONEDAY_BACK = getDateWithOffset(DAILY, -1);
+    System.out.println("Test ONEDAY_BACK:" + ONEDAY_BACK);
+
     TWODAYS_BACK = getDateWithOffset(DAILY, -2);
     System.out.println("Test TWODAYS_BACK:" + TWODAYS_BACK);
 
@@ -214,6 +219,7 @@ public class DateFactory {
     TWO_DAYS_RANGE_TTD2 = getTimeRangeString("test_time_dim2", DAILY, -2, 0, HOURLY);
     TWO_DAYS_RANGE_TTD2_BEFORE_4_DAYS = getTimeRangeString("test_time_dim2", DAILY, -6, -4, HOURLY);
     TWO_DAYS_RANGE_IT = getTimeRangeString("it", DAILY, -2, 0, HOURLY);
+    ONE_DAY_RANGE_IT = getTimeRangeString("it", DAILY, -1, 0, DAILY);
     THIS_YEAR_RANGE = getTimeRangeString(YEARLY, 0, 1);
     LAST_YEAR_RANGE = getTimeRangeString(YEARLY, -1, 0);
     TWO_MONTHS_RANGE_UPTO_MONTH = getTimeRangeString(MONTHLY, -2, 0);

http://git-wip-us.apache.org/repos/asf/lens/blob/ff4f3726/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
index 0366e56..5608b39 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeTestSetup.java
@@ -22,12 +22,7 @@ package org.apache.lens.cube.parse;
 import static java.util.Calendar.DAY_OF_MONTH;
 import static java.util.Calendar.HOUR_OF_DAY;
 
-import static org.apache.lens.cube.metadata.DateFactory.BEFORE_4_DAYS;
-import static org.apache.lens.cube.metadata.DateFactory.BEFORE_6_DAYS;
-import static org.apache.lens.cube.metadata.DateFactory.NOW;
-import static org.apache.lens.cube.metadata.DateFactory.TWODAYS_BACK;
-import static org.apache.lens.cube.metadata.DateFactory.TWO_MONTHS_BACK;
-import static org.apache.lens.cube.metadata.DateFactory.isZerothHour;
+import static org.apache.lens.cube.metadata.DateFactory.*;
 import static org.apache.lens.cube.metadata.UpdatePeriod.DAILY;
 import static org.apache.lens.cube.metadata.UpdatePeriod.HOURLY;
 import static org.apache.lens.cube.metadata.UpdatePeriod.MINUTELY;
@@ -132,6 +127,7 @@ public class CubeTestSetup {
   private static String c3 = "C3";
   private static String c4 = "C4";
   private static String c5 = "C5";
+  private static String c98 = "C98";
   private static String c99 = "C99";
   private static Map<String, String> factValidityProperties = Maps.newHashMap();
   @Getter
@@ -881,10 +877,13 @@ public class CubeTestSetup {
     CubeFactTable fact3 = client.getCubeFactTable(factName);
     createPIEParts(client, fact3, c2);
 
-
     factName = "summary4";
     CubeFactTable fact4 = client.getCubeFactTable(factName);
     createPIEParts(client, fact4, c2);
+
+    factName = "summary5";
+    CubeFactTable fact5 = client.getCubeFactTable(factName);
+    createPIParts(client, fact5, c98);
   }
 
   private void createBaseCubeFactPartitions(CubeMetastoreClient client) throws HiveException, LensException {
@@ -994,6 +993,7 @@ public class CubeTestSetup {
           pTimes.get(HOURLY).add(ptime);
           iTimes.get(HOURLY).add(itime);
           client.addPartition(sPartSpec, storageName, CubeTableType.FACT);
+
           pcal.add(HOUR_OF_DAY, 1);
           ical.add(HOUR_OF_DAY, 1);
         }
@@ -1015,6 +1015,39 @@ public class CubeTestSetup {
     }
   }
 
+
+  private void createPIParts(CubeMetastoreClient client, CubeFactTable fact, String storageName)
+    throws Exception {
+    // Add partitions in PI storage
+    //daily partition registered  for pt=day1, it = day1
+    //hourly partitions registered for pt=day1-hours[0-23] it=day1-hours[0-23]
+    Calendar pcal = Calendar.getInstance();
+    pcal.setTime(ONEDAY_BACK);
+    pcal.set(HOUR_OF_DAY, 0);
+    Calendar ical = Calendar.getInstance();
+    ical.setTime(ONEDAY_BACK);
+    ical.set(HOUR_OF_DAY, 0);
+
+    Date ptime = pcal.getTime();
+    Date itime = ical.getTime();
+    Map<String, Date> timeParts = new HashMap<String, Date>();
+
+    timeParts.put("pt", ptime);
+    timeParts.put("it", itime);
+    StoragePartitionDesc sPartSpec = new StoragePartitionDesc(fact.getName(), timeParts, null, DAILY);
+    client.addPartition(sPartSpec, storageName, CubeTableType.FACT);
+    for (int i = 0; i < 24; i++) {
+      ptime = pcal.getTime();
+      itime = ical.getTime();
+      timeParts.put("pt", ptime);
+      timeParts.put("it", itime);
+      sPartSpec = new StoragePartitionDesc(fact.getName(), timeParts, null, HOURLY);
+      client.addPartition(sPartSpec, storageName, CubeTableType.FACT);
+      pcal.add(HOUR_OF_DAY, 1);
+      ical.add(HOUR_OF_DAY, 1);
+    }
+  }
+
   public static void printQueryAST(String query, String label) throws LensException {
     System.out.println("--" + label + "--AST--");
     System.out.println("--query- " + query);

http://git-wip-us.apache.org/repos/asf/lens/blob/ff4f3726/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 bc0fa2e..9b54957 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
@@ -65,7 +65,7 @@ public class TestCubeRewriter extends TestQueryRewrite {
   @BeforeTest
   public void setupDriver() throws Exception {
     conf = LensServerAPITestUtil.getConfiguration(
-      DRIVER_SUPPORTED_STORAGES, "C0,C1,C2",
+      DRIVER_SUPPORTED_STORAGES, "C0,C1,C2,C98",
       DISABLE_AUTO_JOINS, true,
       ENABLE_SELECT_TO_GROUPBY, true,
       ENABLE_GROUP_BY_TO_SELECT, true,
@@ -1470,6 +1470,19 @@ public class TestCubeRewriter extends TestQueryRewrite {
   }
 
   @Test
+  public void testTimeRangeIn() throws Exception {
+    //check whether time_range_in is resolving in cube rewrite
+    Configuration conf = getConf();
+    conf.set(CubeQueryConfUtil.PROCESS_TIME_PART_COL, "pt");
+    conf.set(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "true");
+    conf.setClass(CubeQueryConfUtil.TIME_RANGE_WRITER_CLASS, AbridgedTimeRangeWriter.class, TimeRangeWriter.class);
+    CubeQueryContext ctx = rewriteCtx("select dim1, sum(msr23)" + " from testCube" + " where " + ONE_DAY_RANGE_IT,
+      conf);
+    String rewrittenQuery = ctx.toHQL();
+    assertTrue(!rewrittenQuery.contains("time_range_in"));
+  }
+
+  @Test
   public void testCubeQueryWithMultipleRanges() throws Exception {
     String hqlQuery =
       rewrite("select SUM(msr2) from testCube" + " where " + TWO_DAYS_RANGE + " OR "

http://git-wip-us.apache.org/repos/asf/lens/blob/ff4f3726/lens-cube/src/test/resources/schema/cubes/base/testcube.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/cubes/base/testcube.xml b/lens-cube/src/test/resources/schema/cubes/base/testcube.xml
index 9d2bb02..4cafd31 100644
--- a/lens-cube/src/test/resources/schema/cubes/base/testcube.xml
+++ b/lens-cube/src/test/resources/schema/cubes/base/testcube.xml
@@ -32,6 +32,8 @@
     <property name="cube.timedim.partition.test_time_dim2" value="ttd2"/>
   </properties>
   <measures>
+    <measure _type="FLOAT" default_aggr="SUM" unit="RS" name="msr23" display_string="Measure23"
+             description="twenty third measure"/>
     <measure _type="FLOAT" default_aggr="SUM" unit="RS" name="msr21" display_string="Measure22"
              description="second measure"/>
     <measure _type="BIGINT" default_aggr="COUNT" name="msr4" display_string="Measure4" description="fourth measure"/>

http://git-wip-us.apache.org/repos/asf/lens/blob/ff4f3726/lens-cube/src/test/resources/schema/facts/summary5.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/summary5.xml b/lens-cube/src/test/resources/schema/facts/summary5.xml
new file mode 100644
index 0000000..a4bbf71
--- /dev/null
+++ b/lens-cube/src/test/resources/schema/facts/summary5.xml
@@ -0,0 +1,70 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+
+  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.
+
+-->
+<x_fact_table name="summary5" cube_name="testCube" weight="15.0" xmlns="uri:lens:cube:0.1">
+  <columns>
+    <column name="msr23" _type="float" comment="twenty third measure"/>
+    <column name="dim1" _type="string" comment="dim1"/>
+    <column name="dim2" _type="string" comment="dim2"/>
+  </columns>
+  <properties>
+    <property name="cube.fact.relative.start.time" value="now.year - 90 days"/>
+    <property name="cube.table.summary5.weight" value="15.0"/>
+    <property name="cube.fact.summary5.storages" value="C98"/>
+    <property name="cube.fact.summary5.c98.updateperiods" value="HOURLY,DAILY"/>
+  </properties>
+  <storage_tables>
+    <storage_table>
+      <storage_name>C98</storage_name>
+      <update_periods>
+        <update_period_table_descriptor>
+          <update_period>HOURLY</update_period>
+          <table_desc external="false">
+            <part_cols>
+              <column name="pt" _type="string" comment="date partition"/>
+              <column name="it" _type="string" comment="date partition"/>
+            </part_cols>
+            <table_parameters>
+              <property name="cube.storagetable.start.times" value="now -4 days"/>
+            </table_parameters>
+            <time_part_cols>pt</time_part_cols>
+            <time_part_cols>it</time_part_cols>
+          </table_desc>
+        </update_period_table_descriptor>
+        <update_period_table_descriptor>
+          <update_period>DAILY</update_period>
+          <table_desc external="false">
+            <part_cols>
+              <column name="pt" _type="string" comment="date partition"/>
+              <column name="it" _type="string" comment="date partition"/>
+            </part_cols>
+            <table_parameters>
+              <property name="cube.storagetable.start.times" value="now.month -3 months"/>
+            </table_parameters>
+            <time_part_cols>pt</time_part_cols>
+            <time_part_cols>it</time_part_cols>
+          </table_desc>
+        </update_period_table_descriptor>
+      </update_periods>
+    </storage_table>
+  </storage_tables>
+</x_fact_table>
+

http://git-wip-us.apache.org/repos/asf/lens/blob/ff4f3726/lens-cube/src/test/resources/schema/storages/c98.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/storages/c98.xml b/lens-cube/src/test/resources/schema/storages/c98.xml
new file mode 100644
index 0000000..550b421
--- /dev/null
+++ b/lens-cube/src/test/resources/schema/storages/c98.xml
@@ -0,0 +1,26 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+
+  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.
+
+-->
+<x_storage name="c98" classname="org.apache.lens.cube.metadata.HDFSStorage" xmlns="uri:lens:cube:0.1">
+  <properties>
+    <property name="cube.table.c98.weight" value="0.0"/>
+  </properties>
+</x_storage>
\ No newline at end of file


[19/50] lens git commit: LENS-1454: Time Covering set algorithm is skipping some combinations

Posted by ra...@apache.org.
LENS-1454: Time Covering set algorithm is skipping some combinations


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

Branch: refs/heads/master
Commit: 8b4120237af9aaca3a6934650ccfeac24bc62c59
Parents: 65ca04b
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Thu Jul 20 16:10:52 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 20 16:13:07 2017 +0530

----------------------------------------------------------------------
 .../lens/cube/parse/CandidateCoveringSetsResolver.java  | 12 +++++++-----
 1 file changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/8b412023/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
index 69d9562..4066cf7 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
@@ -209,21 +209,23 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
   private List<UnionCandidate> getCombinationTailIterative(List<Candidate> candidates, CubeQueryContext cubeql) {
     LinkedList<Candidate> candidateLinkedList = Lists.newLinkedList(candidates);
     List<List<Candidate>> incompleteCombinations = Lists.newArrayList();
+    incompleteCombinations.add(Lists.newArrayList());
     List<UnionCandidate> unionCandidates = Lists.newArrayList();
 
     while(!candidateLinkedList.isEmpty()) {
+      List<List<Candidate>> moreIncomplete = Lists.newArrayList();
       Candidate candidate = candidateLinkedList.remove();
-      incompleteCombinations.add(Lists.newArrayList());
-      Iterator<List<Candidate>> iter = incompleteCombinations.iterator();
-      while(iter.hasNext()) {
-        List<Candidate> incompleteCombination = iter.next();
+      for (List<Candidate> combination : incompleteCombinations) {
+        List<Candidate> incompleteCombination = Lists.newArrayList(combination);
         incompleteCombination.add(candidate);
         UnionCandidate unionCandidate = new UnionCandidate(incompleteCombination, cubeql);
         if (isCandidateCoveringTimeRanges(unionCandidate, cubeql.getTimeRanges())) {
           unionCandidates.add(unionCandidate);
-          iter.remove();
+        } else {
+          moreIncomplete.add(incompleteCombination);
         }
       }
+      incompleteCombinations.addAll(moreIncomplete);
     }
     return unionCandidates;
   }


[23/50] lens git commit: Revert "LENS-1450: filters are getting replicated in jdbc driver query"

Posted by ra...@apache.org.
Revert "LENS-1450: filters are getting replicated in jdbc driver query"

This reverts commit 63e0b691741b9ac045cbfe6dd988000988077800.


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

Branch: refs/heads/master
Commit: 1f9f12f25b26ff00443410c7318e785499cd19f6
Parents: f43c205
Author: Rajat Khandelwal <ra...@gmail.com>
Authored: Wed Aug 2 20:47:38 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Wed Aug 2 20:47:53 2017 +0530

----------------------------------------------------------------------
 .../src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/1f9f12f2/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 fb49b6d..b5330a7 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
@@ -1169,7 +1169,7 @@ public class CubeQueryContext extends TracksQueriedColumns implements QueryAST,
         }
       }
     } else if (node.getParent() == null
-        && node.getToken().getType() != HiveParser.KW_AND && node.getChildCount() == 0) {
+        && node.getToken().getType() != HiveParser.KW_AND) {
       // if node is the only child
       allFilters.add(HQLParser.getString((ASTNode) node));
     }


[41/50] lens git commit: LENS-1492: Update cli.apt file

Posted by ra...@apache.org.
LENS-1492: Update cli.apt file


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

Branch: refs/heads/master
Commit: 8f9b5686e269c9318321fd766fccc37d387a110c
Parents: 7fa91dd
Author: rajub <ra...@lazada.com>
Authored: Mon Dec 4 21:22:00 2017 +0800
Committer: rajub <ra...@lazada.com>
Committed: Mon Dec 4 21:22:00 2017 +0800

----------------------------------------------------------------------
 src/site/apt/user/cli.apt | 92 ++++++++++++++++++++++--------------------
 1 file changed, 49 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/8f9b5686/src/site/apt/user/cli.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/user/cli.apt b/src/site/apt/user/cli.apt
index 1653e9f..12bc88c 100644
--- a/src/site/apt/user/cli.apt
+++ b/src/site/apt/user/cli.apt
@@ -389,49 +389,55 @@ User CLI Commands
 *--+--+
 |<<Command>>|<<Description>>|
 *--+--+
-|schema/create schema [--db] \<database-to-create-schema-in\> [--path/--file] \<schema-directory\>|Parses the specified resource file and executes commands for creation/updation of schema                                       \ |
-|                                                                                                 |Expected\ structure\ is\                                                                                                       \ |
-|                                                                                                 |.                                                                                                                              \ |
-|                                                                                                 |\|--\ storages                                                                                                                 \ |
-|                                                                                                 |\|\ \ \|--\ storage1.xml                                                                                                       \ |
-|                                                                                                 |\|\ \ \|--\ storage2.xml                                                                                                       \ |
-|                                                                                                 |\|                                                                                                                             \ |
-|                                                                                                 |\|--\ dimensions                                                                                                               \ |
-|                                                                                                 |\|\ \ \|--\ dim1.xml                                                                                                           \ |
-|                                                                                                 |\|\ \ \|--\ dim2.xml                                                                                                           \ |
-|                                                                                                 |\|                                                                                                                             \ |
-|                                                                                                 |\|--\ cubes                                                                                                                    \ |
-|                                                                                                 |\|\ \ \|--\ base                                                                                                               \ |
-|                                                                                                 |\|\ \ \|\ \ \|--\ base_cube1.xml                                                                                               \ |
-|                                                                                                 |\|\ \ \|\ \ \|--\ base_cube2.xml                                                                                               \ |
-|                                                                                                 |\|\ \ \|                                                                                                                       \ |
-|                                                                                                 |\|\ \ \|--\ derived                                                                                                            \ |
-|                                                                                                 |\|\ \ \|\ \ \|--\ derived_cube1.xml                                                                                            \ |
-|                                                                                                 |\|\ \ \|\ \ \|--\ derived_cube2.xml                                                                                            \ |
-|                                                                                                 |\|\ \ \|                                                                                                                       \ |
-|                                                                                                 |\|\ \ \|--\ independent_cube1.xml                                                                                              \ |
-|                                                                                                 |\|\ \ \|--\ independent_cube2.xml                                                                                              \ |
-|                                                                                                 |\|                                                                                                                             \ |
-|                                                                                                 |\|--\ dimensiontables                                                                                                          \ |
-|                                                                                                 |\|\ \ \|--\ dimtable1.xml                                                                                                      \ |
-|                                                                                                 |\|\ \ \|--\ dimtable2.xml                                                                                                      \ |
-|                                                                                                 |\|                                                                                                                             \ |
-|                                                                                                 |\|--\ dimtables                                                                                                                \ |
-|                                                                                                 |\|\ \ \|--\ dimtable3.xml                                                                                                      \ |
-|                                                                                                 |\|\ \ \|--\ dimtable4.xml                                                                                                      \ |
-|                                                                                                 |\|                                                                                                                             \ |
-|                                                                                                 |\|--\ facts                                                                                                                    \ |
-|                                                                                                 |\ \ \ \|--\ fact1.xml                                                                                                          \ |
-|                                                                                                 |\ \ \ \|--\ fact2.xml                                                                                                          \ |
-|                                                                                                 |                                                                                                                               \ |
-|                                                                                                 |                                                                                                                               \ |
-|                                                                                                 |If\ your\ cubes\ are\ divided\ between\ base\ and\ derived\ cubes,                                                             \ |
-|                                                                                                 |it\ makes\ sense\ to\ seperate\ into\ two\ directories,\ since\ derived\ cubes\ can't\ be\ created\ unless\ base\ cube\ exists.\ |
-|                                                                                                 |In\ the\ other\ case\ you\ can\ keep\ them\ in\ the\ cubes\ directory\ itself.                                                 \ |
-|                                                                                                 |For\ dimtables,\ you\ can\ keep\ your\ schema\ files\ in\ a\ directory\ named\ either\ dimtables\ or\ dimensiontables.         \ |
-|                                                                                                 |Each\ of\ these\ directories\ is\ optional\ and\ the\ order\ of\ processing\ is\ top\ to\ bottom.                              \ |
-|                                                                                                 |CLI\ will\ let\ you\ know\ in\ case\ of\ any\ errors\ and\ proceed\ further\ without\ failing\ in\ between.                    \ |
+|schema/create schema [--db] \<database-to-create-schema-in\> [--path/--file] \<schema-directory\> [[--type] \<schema-type-filter\>] [[--name] \<file-name-filter\>]|Parses the specified resource file and executes commands for creation/updation of schema. If <<<schema-type-filter>>> is provided, only schema types matching that will be worked upon. If <<<file-name-filter>>> is provided, then only those files that contain the filter value will be worked upon. \ |
+|                                                                                                                                                                   |Expected\ directory\ structure\ is\                                                                                                                                                                                                                                                                     \ |
+|                                                                                                                                                                   |.                                                                                                                                                                                                                                                                                                       \ |
+|                                                                                                                                                                   |\|--\ storages                                                                                                                                                                                                                                                                                          \ |
+|                                                                                                                                                                   |\|\ \ \|--\ storage1.xml                                                                                                                                                                                                                                                                                \ |
+|                                                                                                                                                                   |\|\ \ \|--\ storage2.xml                                                                                                                                                                                                                                                                                \ |
+|                                                                                                                                                                   |\|                                                                                                                                                                                                                                                                                                      \ |
+|                                                                                                                                                                   |\|--\ dimensions                                                                                                                                                                                                                                                                                        \ |
+|                                                                                                                                                                   |\|\ \ \|--\ dim1.xml                                                                                                                                                                                                                                                                                    \ |
+|                                                                                                                                                                   |\|\ \ \|--\ dim2.xml                                                                                                                                                                                                                                                                                    \ |
+|                                                                                                                                                                   |\|                                                                                                                                                                                                                                                                                                      \ |
+|                                                                                                                                                                   |\|--\ cubes                                                                                                                                                                                                                                                                                             \ |
+|                                                                                                                                                                   |\|\ \ \|--\ base                                                                                                                                                                                                                                                                                        \ |
+|                                                                                                                                                                   |\|\ \ \|\ \ \|--\ base_cube1.xml                                                                                                                                                                                                                                                                        \ |
+|                                                                                                                                                                   |\|\ \ \|\ \ \|--\ base_cube2.xml                                                                                                                                                                                                                                                                        \ |
+|                                                                                                                                                                   |\|\ \ \|                                                                                                                                                                                                                                                                                                \ |
+|                                                                                                                                                                   |\|\ \ \|--\ derived                                                                                                                                                                                                                                                                                     \ |
+|                                                                                                                                                                   |\|\ \ \|\ \ \|--\ derived_cube1.xml                                                                                                                                                                                                                                                                     \ |
+|                                                                                                                                                                   |\|\ \ \|\ \ \|--\ derived_cube2.xml                                                                                                                                                                                                                                                                     \ |
+|                                                                                                                                                                   |\|\ \ \|                                                                                                                                                                                                                                                                                                \ |
+|                                                                                                                                                                   |\|\ \ \|--\ independent_cube1.xml                                                                                                                                                                                                                                                                       \ |
+|                                                                                                                                                                   |\|\ \ \|--\ independent_cube2.xml                                                                                                                                                                                                                                                                       \ |
+|                                                                                                                                                                   |\|                                                                                                                                                                                                                                                                                                      \ |
+|                                                                                                                                                                   |\|--\ dimensiontables                                                                                                                                                                                                                                                                                   \ |
+|                                                                                                                                                                   |\|\ \ \|--\ dimtable1.xml                                                                                                                                                                                                                                                                               \ |
+|                                                                                                                                                                   |\|\ \ \|--\ dimtable2.xml                                                                                                                                                                                                                                                                               \ |
+|                                                                                                                                                                   |\|                                                                                                                                                                                                                                                                                                      \ |
+|                                                                                                                                                                   |\|--\ dimtables                                                                                                                                                                                                                                                                                         \ |
+|                                                                                                                                                                   |\|\ \ \|--\ dimtable3.xml                                                                                                                                                                                                                                                                               \ |
+|                                                                                                                                                                   |\|\ \ \|--\ dimtable4.xml                                                                                                                                                                                                                                                                               \ |
+|                                                                                                                                                                   |\|                                                                                                                                                                                                                                                                                                      \ |
+|                                                                                                                                                                   |\|--\ facts                                                                                                                                                                                                                                                                                             \ |
+|                                                                                                                                                                   |\ \ \ \|--\ fact1.xml                                                                                                                                                                                                                                                                                   \ |
+|                                                                                                                                                                   |\ \ \ \|--\ fact2.xml                                                                                                                                                                                                                                                                                   \ |
+|                                                                                                                                                                   |\|\ \ \|                                                                                                                                                                                                                                                                                                \ |
+|                                                                                                                                                                   |\|\ \ \|--\ virtual                                                                                                                                                                                                                                                                                     \ |
+|                                                                                                                                                                   |\|\ \ \|\ \ \|--\ virtual_fact1.xml                                                                                                                                                                                                                                                                     \ |
+|                                                                                                                                                                   |\|\ \ \|\ \ \|--\ virtual_fact2.xml                                                                                                                                                                                                                                                                     \ |
+|                                                                                                                                                                   |\|\ \ \|                                                                                                                                                                                                                                                                                                \ |
+|                                                                                                                                                                   |                                                                                                                                                                                                                                                                                                        \ |
+|                                                                                                                                                                   |                                                                                                                                                                                                                                                                                                        \ |
+|                                                                                                                                                                   |If\ your\ cubes\ are\ divided\ between\ base\ and\ derived\ cubes,                                                                                                                                                                                                                                      \ |
+|                                                                                                                                                                   |it\ makes\ sense\ to\ seperate\ into\ two\ directories,\ since\ derived\ cubes\ can't\ be\ created\ unless\ base\ cube\ exists.                                                                                                                                                                         \ |
+|                                                                                                                                                                   |In\ the\ other\ case\ you\ can\ keep\ them\ in\ the\ cubes\ directory\ itself.                                                                                                                                                                                                                          \ |
+|                                                                                                                                                                   |For\ dimtables,\ you\ can\ keep\ your\ schema\ files\ in\ a\ directory\ named\ either\ dimtables\ or\ dimensiontables.                                                                                                                                                                                  \ |
+|                                                                                                                                                                   |Each\ of\ these\ directories\ is\ optional\ and\ the\ order\ of\ processing\ is\ top\ to\ bottom.                                                                                                                                                                                                       \ |
+|                                                                                                                                                                   |CLI\ will\ let\ you\ know\ in\ case\ of\ any\ errors\ and\ proceed\ further\ without\ failing\ in\ between.                                                                                                                                                                                             \ |
 *--+--+
   <<Lens Schema Commands>>
 
-===
\ No newline at end of file
+===
+


[08/50] lens git commit: LENS-1442: Optimize algorithm of CandidateCoveringSetResolver

Posted by ra...@apache.org.
LENS-1442: Optimize algorithm of CandidateCoveringSetResolver


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

Branch: refs/heads/master
Commit: 9da5b40e3afbe966256a601bae6204afd5d3e992
Parents: c2a9c93
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Tue Jun 20 16:09:32 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:42:49 2017 +0530

----------------------------------------------------------------------
 .../cube/parse/CandidateCoveringSetsResolver.java    | 15 +++++++++------
 1 file changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/9da5b40e/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
index 1e9873f..61c28c6 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
@@ -208,21 +208,24 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
     List<Candidate> ucSet = new ArrayList<>(candidates);
     // Check if a single set can answer all the measures and exprsWithMeasures
     for (Iterator<Candidate> i = ucSet.iterator(); i.hasNext();) {
-      boolean evaluable = false;
+      boolean allEvaluable = true;
+      boolean anyEvaluable = false;
       Candidate uc = i.next();
       for (QueriedPhraseContext msr : msrs) {
-        evaluable = uc.isPhraseAnswerable(msr);
-        if (!evaluable) {
-          break;
-        }
+        boolean evaluable = uc.isPhraseAnswerable(msr);
+        allEvaluable &= evaluable;
+        anyEvaluable |= evaluable;
       }
-      if (evaluable) {
+      if (allEvaluable) {
         // single set can answer all the measures as an UnionCandidate
         List<Candidate> one = new ArrayList<>();
         one.add(uc);
         msrCoveringSets.add(one);
         i.remove();
       }
+      if (!anyEvaluable) { // none evaluable
+        i.remove();
+      }
     }
     // Sets that contain all measures or no measures are removed from iteration.
     // find other facts


[13/50] lens git commit: LENS-1448: Having clause expressions are not resolved correctly for JoinCandidates

Posted by ra...@apache.org.
LENS-1448: Having clause expressions are not resolved correctly for JoinCandidates


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

Branch: refs/heads/master
Commit: 54ab131a2119a029dd8a470653d803fa0a7c35b6
Parents: fbad350
Author: Sushil Mohanty <su...@gmail.com>
Authored: Wed Jun 28 13:10:55 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:42:55 2017 +0530

----------------------------------------------------------------------
 .../lens/cube/parse/ExpressionResolver.java     | 11 +++++------
 .../lens/cube/parse/UnionQueryWriter.java       | 19 +++++++++++--------
 .../lens/cube/parse/TestBaseCubeQueries.java    | 20 ++++++++++++++++++++
 3 files changed, 36 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/54ab131a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
index 2403576..66b043e 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
@@ -424,20 +424,19 @@ class ExpressionResolver implements ContextRewriter {
       }
       replaceAST(cubeql, queryAST.getJoinAST());
       replaceAST(cubeql, queryAST.getGroupByAST());
-      // Having AST is resolved by each fact, so that all facts can expand their expressions.
-      // Having ast is not copied now, it's maintained in cubeQueryContext, each fact processes that serially.
+      // Resolve having expression for StorageCandidate
       if (queryAST.getHavingAST() != null) {
         replaceAST(cubeql, queryAST.getHavingAST());
-      } else if (cubeql.getHavingAST() != null && nonPickedExpressionsForCandidate.isEmpty()) {
-        replaceAST(cubeql, cubeql.getHavingAST());
-        queryAST.setHavingAST(MetastoreUtil.copyAST(cubeql.getHavingAST()));
+      } else if (cubeql.getHavingAST() != null) {
+        ASTNode havingCopy = MetastoreUtil.copyAST(cubeql.getHavingAST());
+        replaceAST(cubeql, havingCopy);
+        queryAST.setHavingAST(havingCopy);
       }
       replaceAST(cubeql, queryAST.getOrderByAST());
     }
 
     private void replaceAST(final CubeQueryContext cubeql, ASTNode node) throws LensException {
       if (node == null) {
-
         return;
       }
       // Traverse the tree and resolve expression columns

http://git-wip-us.apache.org/repos/asf/lens/blob/54ab131a/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
index f6c9ce1..9dc7ee6 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
@@ -96,7 +96,6 @@ public class UnionQueryWriter extends SimpleHQLContext {
       if (sc.getQueryAst().getHavingAST() != null) {
         cubeql.setHavingAST(sc.getQueryAst().getHavingAST());
       }
-      sc.getQueryAst().setHavingAST(null);
       sc.getQueryAst().setOrderByAST(null);
       sc.getQueryAst().setLimitValue(null);
     }
@@ -120,8 +119,12 @@ public class UnionQueryWriter extends SimpleHQLContext {
    */
   private ASTNode processHavingAST(ASTNode innerAst, AliasDecider aliasDecider, StorageCandidateHQLContext sc)
     throws LensException {
-    if (cubeql.getHavingAST() != null) {
-      ASTNode havingCopy = MetastoreUtil.copyAST(cubeql.getHavingAST());
+    if (sc.getQueryAst().getHavingAST() == null
+        && cubeql.getHavingAST() != null) {
+      sc.getQueryAst().setHavingAST(cubeql.getHavingAST());
+    }
+    if (sc.getQueryAst().getHavingAST() != null) {
+      ASTNode havingCopy = MetastoreUtil.copyAST(sc.getQueryAst().getHavingAST());
       Set<ASTNode> havingAggChildrenASTs = new LinkedHashSet<>();
       getAggregateChildrenInNode(havingCopy, havingAggChildrenASTs);
       processHavingExpression(innerAst, havingAggChildrenASTs, aliasDecider, sc);
@@ -283,8 +286,9 @@ public class UnionQueryWriter extends SimpleHQLContext {
   }
 
   private boolean isNodeDefault(ASTNode node) {
-    if (HQLParser.isAggregateAST((ASTNode) node.getChild(0))) {
-      if (HQLParser.getString((ASTNode) node.getChild(0).getChild(1)).equals(DEFAULT_MEASURE)) {
+    if (HQLParser.isAggregateAST((ASTNode) node.getChild(0)) || HQLParser.isAggregateAST(node)) {
+      if (HQLParser.getString((ASTNode) node.getChild(0).getChild(1)).equals(DEFAULT_MEASURE)
+          || HQLParser.getString((ASTNode) node.getChild(1)).equals(DEFAULT_MEASURE)) {
         return true;
       }
     }
@@ -640,9 +644,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
       AliasDecider aliasDecider, StorageCandidateHQLContext sc) throws LensException {
     // iterate over all children of the ast and get outer ast corresponding to it.
     for (ASTNode child : havingAggASTs) {
-      if (!innerToOuterSelectASTs.containsKey(new HQLParser.HashableASTNode(child))) {
-        getOuterAST(child, innerSelectAst, aliasDecider, sc, false, cubeql.getBaseCube().getDimAttributeNames());
-      }
+      getOuterAST(child, innerSelectAst, aliasDecider, sc, false, cubeql.getBaseCube().getDimAttributeNames());
     }
   }
 
@@ -726,6 +728,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
     List<String> hqlQueries = new ArrayList<>();
     for (StorageCandidateHQLContext sc : storageCandidates) {
       removeAggreagateFromDefaultColumns(sc.getQueryAst().getSelectAST());
+      sc.getQueryAst().setHavingAST(null);
       hqlQueries.add(sc.toHQL());
     }
     return hqlQueries.stream().collect(joining(" UNION ALL ", "(", ") as " + cubeql.getBaseCube()));

http://git-wip-us.apache.org/repos/asf/lens/blob/54ab131a/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 f87158c..cf29dff 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
@@ -882,6 +882,26 @@ public class TestBaseCubeQueries extends TestQueryRewrite {
     assertTrue(hqlQuery.endsWith("HAVING ((sum((basecube.alias2)) > 2) "
         + "and (round((sum((basecube.alias3)) / 1000)) > 0))"));
 
+    // Having clause with expression answerable from any one fact and not projected
+    hqlQuery = rewrite("select dim1, dim11, msr12  from basecube where " + TWO_DAYS_RANGE
+        + "having msr12 > 2 and roundedmsr2 > 0", conf);
+    expected1 = getExpectedQuery(cubeName,
+        "SELECT (basecube.dim1) as `alias0`, (basecube.dim11) as `alias1`, sum((basecube.msr12)) "
+            + "as `alias2`, 0.0 as `alias4` FROM ", null, " group by basecube.dim1, basecube.dim11",
+        getWhereForDailyAndHourly2days(cubeName, "C1_testFact2_BASE"));
+    expected2 = getExpectedQuery(cubeName,
+        "SELECT (basecube.dim1) as `alias0`, (basecube.dim11) as `alias1`, 0.0 as `alias2`, "
+            + "sum((basecube.msr2)) as `alias4` FROM ", null, " group by basecube.dim1, basecube.dim11",
+        getWhereForDailyAndHourly2days(cubeName, "C1_testFact1_BASE"));
+
+    compareContains(expected1, hqlQuery);
+    compareContains(expected2, hqlQuery);
+    assertTrue(hqlQuery.toLowerCase().startsWith("select (basecube.alias0) as `dim1`, (basecube.alias1) as `dim11`, "
+        + "sum((basecube.alias2)) as `msr12` from"),
+        hqlQuery);
+    assertTrue(hqlQuery.endsWith("HAVING ((sum((basecube.alias2)) > 2) and "
+        + "(round((sum((basecube.alias4)) / 1000)) > 0))"));
+
     hqlQuery = rewrite("select dim1, dim11, msr12, roundedmsr2 from basecube where " + TWO_DAYS_RANGE
         + "having msr12+roundedmsr2 <= 1000 and msr12 > 2 and roundedmsr2 > 0", conf);
     expected1 = getExpectedQuery(cubeName,


[22/50] lens git commit: LENS-1457: ExpressionResolver pruning expression when some tables don't have ref columns of the expressions

Posted by ra...@apache.org.
LENS-1457: ExpressionResolver pruning expression when some tables don't have ref columns of the expressions


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

Branch: refs/heads/master
Commit: f43c20512facb9686316b5b4aed1af9515d33fb5
Parents: cdd7b09
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Mon Jul 31 15:16:35 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Wed Aug 2 20:47:04 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/lens/cube/parse/CandidateTable.java   | 10 ++--------
 .../org/apache/lens/cube/parse/ExpressionResolver.java    |  4 ++++
 2 files changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/f43c2051/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTable.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTable.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTable.java
index c909545..40022f0 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTable.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTable.java
@@ -22,11 +22,12 @@ import java.util.Collection;
 import java.util.Set;
 
 import org.apache.lens.cube.metadata.AbstractCubeTable;
+import org.apache.lens.cube.metadata.Named;
 
 /**
  * Candidate table interface
  */
-public interface CandidateTable {
+public interface CandidateTable extends Named {
 
   /**
    * Get storage string of the base table alias passed
@@ -57,13 +58,6 @@ public interface CandidateTable {
   AbstractCubeTable getBaseTable();
 
   /**
-   * Get name of the candidate table
-   *
-   * @return name
-   */
-  String getName();
-
-  /**
    * Get columns of candidate table
    *
    * @return set or list of columns

http://git-wip-us.apache.org/repos/asf/lens/blob/f43c2051/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
index 4680766..8906fae 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
@@ -19,6 +19,8 @@
 
 package org.apache.lens.cube.parse;
 
+import static java.util.stream.Collectors.toSet;
+
 import static org.apache.hadoop.hive.ql.parse.HiveParser.*;
 
 import java.util.*;
@@ -557,6 +559,8 @@ class ExpressionResolver implements ContextRewriter {
             // Remove expressions for which denormalized columns are no more reachable
             esc.getDeNormCtx().pruneReferences(cubeql);
             if (!esc.getDeNormCtx().getTableToRefCols().isEmpty()
+              && esc.getDeNormCtx().getTableToRefCols().keySet().containsAll(
+                ec.getEvaluableExpressions().keySet().stream().map(Named::getName).collect(toSet()))
               && esc.getDeNormCtx().getTableToRefCols().keySet().stream()
               .map(esc.getDeNormCtx()::getNonReachableReferenceFields).noneMatch(Set::isEmpty)) {
               log.info("Removing expression {} as all tables have non reachable fields", esc);


[48/50] lens git commit: [maven-release-plugin] prepare release apache-lens-2.7.1

Posted by ra...@apache.org.
[maven-release-plugin] prepare release apache-lens-2.7.1


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

Branch: refs/heads/master
Commit: 81147a54fded8407ccedd04a925760b51af4637a
Parents: 165c354
Author: rajub <ra...@lazada.com>
Authored: Wed Jan 24 09:16:51 2018 +0800
Committer: rajub <ra...@lazada.com>
Committed: Wed Jan 24 09:16:51 2018 +0800

----------------------------------------------------------------------
 checkstyle/pom.xml             | 6 +++++-
 contrib/clients/pom.xml        | 2 +-
 contrib/clients/python/pom.xml | 2 +-
 contrib/pom.xml                | 2 +-
 lens-api/pom.xml               | 2 +-
 lens-cli/pom.xml               | 2 +-
 lens-client/pom.xml            | 2 +-
 lens-cube/pom.xml              | 2 +-
 lens-dist/pom.xml              | 2 +-
 lens-driver-es/pom.xml         | 2 +-
 lens-driver-hive/pom.xml       | 2 +-
 lens-driver-jdbc/pom.xml       | 2 +-
 lens-examples/pom.xml          | 2 +-
 lens-ml-dist/pom.xml           | 2 +-
 lens-ml-lib/pom.xml            | 2 +-
 lens-query-lib/pom.xml         | 2 +-
 lens-regression/pom.xml        | 2 +-
 lens-server-api/pom.xml        | 2 +-
 lens-server/pom.xml            | 2 +-
 lens-ship-jars/pom.xml         | 2 +-
 lens-storage-db/pom.xml        | 2 +-
 lens-ui/pom.xml                | 2 +-
 pom.xml                        | 8 ++++----
 23 files changed, 30 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/checkstyle/pom.xml
----------------------------------------------------------------------
diff --git a/checkstyle/pom.xml b/checkstyle/pom.xml
index 0076790..a46b4a1 100644
--- a/checkstyle/pom.xml
+++ b/checkstyle/pom.xml
@@ -29,7 +29,7 @@
   <groupId>org.apache.lens</groupId>
   <artifactId>checkstyle</artifactId>
   <name>Lens Checkstyle Rules</name>
-  <version>2.7.1-SNAPSHOT</version>
+  <version>2.7.1</version>
 
   <build>
     <plugins>
@@ -49,4 +49,8 @@
       </plugin>
     </plugins>
   </build>
+
+  <scm>
+    <tag>apache-lens-2.7.1</tag>
+  </scm>
 </project>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/contrib/clients/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/clients/pom.xml b/contrib/clients/pom.xml
index 260cac7..c648626 100644
--- a/contrib/clients/pom.xml
+++ b/contrib/clients/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>lens-contrib</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-contrib-clients</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/contrib/clients/python/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/clients/python/pom.xml b/contrib/clients/python/pom.xml
index 039d29d..c233ce5 100644
--- a/contrib/clients/python/pom.xml
+++ b/contrib/clients/python/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>lens-contrib-clients</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-python-client</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/contrib/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/pom.xml b/contrib/pom.xml
index d3d14e8..cd44fa6 100644
--- a/contrib/pom.xml
+++ b/contrib/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-contrib</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-api/pom.xml
----------------------------------------------------------------------
diff --git a/lens-api/pom.xml b/lens-api/pom.xml
index b36e3c5..58ab1be 100644
--- a/lens-api/pom.xml
+++ b/lens-api/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-api</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-cli/pom.xml
----------------------------------------------------------------------
diff --git a/lens-cli/pom.xml b/lens-cli/pom.xml
index b09d826..f9e758c 100644
--- a/lens-cli/pom.xml
+++ b/lens-cli/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-client/pom.xml
----------------------------------------------------------------------
diff --git a/lens-client/pom.xml b/lens-client/pom.xml
index 1591fb2..9af3903 100644
--- a/lens-client/pom.xml
+++ b/lens-client/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-cube/pom.xml
----------------------------------------------------------------------
diff --git a/lens-cube/pom.xml b/lens-cube/pom.xml
index 7a7cb03..074a593 100644
--- a/lens-cube/pom.xml
+++ b/lens-cube/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-cube</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-dist/pom.xml
----------------------------------------------------------------------
diff --git a/lens-dist/pom.xml b/lens-dist/pom.xml
index 22b6ca5..10eb7c3 100644
--- a/lens-dist/pom.xml
+++ b/lens-dist/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-dist</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-driver-es/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-es/pom.xml b/lens-driver-es/pom.xml
index 5b3f61f..e20505b 100644
--- a/lens-driver-es/pom.xml
+++ b/lens-driver-es/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache.lens</groupId>
     <artifactId>apache-lens</artifactId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-driver-es</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-driver-hive/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-hive/pom.xml b/lens-driver-hive/pom.xml
index 08ecff2..a3ad080 100644
--- a/lens-driver-hive/pom.xml
+++ b/lens-driver-hive/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-driver-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-jdbc/pom.xml b/lens-driver-jdbc/pom.xml
index d2164f2..22d1095 100644
--- a/lens-driver-jdbc/pom.xml
+++ b/lens-driver-jdbc/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-driver-jdbc</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-examples/pom.xml
----------------------------------------------------------------------
diff --git a/lens-examples/pom.xml b/lens-examples/pom.xml
index c9c11a8..1af8fcf 100644
--- a/lens-examples/pom.xml
+++ b/lens-examples/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-ml-dist/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ml-dist/pom.xml b/lens-ml-dist/pom.xml
index 8a744e5..15bc2a2 100644
--- a/lens-ml-dist/pom.xml
+++ b/lens-ml-dist/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
     
   <artifactId>lens-ml-dist</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-ml-lib/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ml-lib/pom.xml b/lens-ml-lib/pom.xml
index cc587bb..58f9844 100644
--- a/lens-ml-lib/pom.xml
+++ b/lens-ml-lib/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-query-lib/pom.xml
----------------------------------------------------------------------
diff --git a/lens-query-lib/pom.xml b/lens-query-lib/pom.xml
index 558553d..838dac4 100644
--- a/lens-query-lib/pom.xml
+++ b/lens-query-lib/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-query-lib</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-regression/pom.xml
----------------------------------------------------------------------
diff --git a/lens-regression/pom.xml b/lens-regression/pom.xml
index 7cde8f0..8aeed6b 100644
--- a/lens-regression/pom.xml
+++ b/lens-regression/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-regression</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-server-api/pom.xml
----------------------------------------------------------------------
diff --git a/lens-server-api/pom.xml b/lens-server-api/pom.xml
index 5ac73da..347ae39 100644
--- a/lens-server-api/pom.xml
+++ b/lens-server-api/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-server-api</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-server/pom.xml
----------------------------------------------------------------------
diff --git a/lens-server/pom.xml b/lens-server/pom.xml
index 0d99138..1badfb0 100644
--- a/lens-server/pom.xml
+++ b/lens-server/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-ship-jars/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ship-jars/pom.xml b/lens-ship-jars/pom.xml
index 1d3cd6e..9725434 100644
--- a/lens-ship-jars/pom.xml
+++ b/lens-ship-jars/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-ship-jars</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-storage-db/pom.xml
----------------------------------------------------------------------
diff --git a/lens-storage-db/pom.xml b/lens-storage-db/pom.xml
index 544ef29..1cb6e09 100644
--- a/lens-storage-db/pom.xml
+++ b/lens-storage-db/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-storage-db</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/lens-ui/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ui/pom.xml b/lens-ui/pom.xml
index 55320c6..12a2136 100644
--- a/lens-ui/pom.xml
+++ b/lens-ui/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1-SNAPSHOT</version>
+    <version>2.7.1</version>
   </parent>
 
   <artifactId>lens-ui</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/81147a54/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a50aed3..b43a2fa 100644
--- a/pom.xml
+++ b/pom.xml
@@ -128,7 +128,7 @@
 
   <groupId>org.apache.lens</groupId>
   <artifactId>apache-lens</artifactId>
-  <version>2.7.1-SNAPSHOT</version>
+  <version>2.7.1</version>
   <name>Lens</name>
   <packaging>pom</packaging>
   <description>Unified Analytics Platform</description>
@@ -379,7 +379,7 @@
     <connection>scm:git:https://git-wip-us.apache.org/repos/asf/lens.git</connection>
     <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/lens.git</developerConnection>
     <url>https://git-wip-us.apache.org/repos/asf/lens.git</url>
-    <tag>HEAD</tag>
+    <tag>apache-lens-2.7.1</tag>
   </scm>
 
   <reporting>
@@ -740,7 +740,7 @@
           <dependency>
             <groupId>org.apache.lens</groupId>
             <artifactId>checkstyle</artifactId>
-            <version>2.7.1-SNAPSHOT</version>
+            <version>2.7.1</version>
           </dependency>
         </dependencies>
         <executions>
@@ -1854,7 +1854,7 @@
               <dependency>
                 <groupId>org.apache.lens</groupId>
                 <artifactId>checkstyle</artifactId>
-                <version>2.7.1-SNAPSHOT</version>
+                <version>2.7.1</version>
               </dependency>
             </dependencies>
             <executions>


[11/50] lens git commit: LENS-1445: Expression having reference column ends up rewriting wrong query

Posted by ra...@apache.org.
LENS-1445: Expression having reference column ends up rewriting wrong query


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

Branch: refs/heads/master
Commit: d49f45a0f8c6665784a3770a534d6495c21fd1bc
Parents: a7f407b
Author: Sushil Mohanty <su...@gmail.com>
Authored: Fri Jun 23 16:40:25 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:42:53 2017 +0530

----------------------------------------------------------------------
 .../lens/cube/parse/UnionQueryWriter.java       | 79 +++++++++++++-------
 .../parse/TestCubeSegmentationRewriter.java     |  3 +-
 .../cube/parse/TestUnionAndJoinCandidates.java  | 26 +++++++
 .../resources/schema/cubes/base/basecube.xml    | 11 +++
 .../resources/schema/cubes/base/testcube.xml    |  1 +
 5 files changed, 92 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/d49f45a0/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
index cc0a2e5..f6c9ce1 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
@@ -28,7 +28,8 @@ import static org.apache.hadoop.hive.ql.parse.HiveParser.*;
 
 import java.util.*;
 
-import org.apache.lens.cube.metadata.MetastoreUtil;
+import org.apache.lens.cube.metadata.*;
+import org.apache.lens.cube.metadata.join.JoinPath;
 import org.apache.lens.server.api.error.LensException;
 
 import org.apache.hadoop.hive.ql.lib.Node;
@@ -117,7 +118,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
    * @return ASTNode
    * @throws LensException
    */
-  private ASTNode processHavingAST(ASTNode innerAst, AliasDecider aliasDecider, StorageCandidate sc)
+  private ASTNode processHavingAST(ASTNode innerAst, AliasDecider aliasDecider, StorageCandidateHQLContext sc)
     throws LensException {
     if (cubeql.getHavingAST() != null) {
       ASTNode havingCopy = MetastoreUtil.copyAST(cubeql.getHavingAST());
@@ -251,25 +252,15 @@ public class UnionQueryWriter extends SimpleHQLContext {
     for (StorageCandidateHQLContext sc : storageCandidates) {
       node = (ASTNode) sc.getQueryAst().getSelectAST().getChild(position).getChild(0);
       if (HQLParser.isAggregateAST(node) || HQLParser.hasAggregate(node)) {
-        return MetastoreUtil.copyAST(node);
+        if (!node.getChild(1).toString().equals(DEFAULT_MEASURE))  {
+          return MetastoreUtil.copyAST(node);
+        }
       }
     }
     return MetastoreUtil.copyAST(node);
   }
 
   /**
-   *  Check if ASTNode is answerable by StorageCandidate
-   * @param sc
-   * @param node
-   * @return
-   */
-  private boolean isNodeNotAnswerableForStorageCandidate(StorageCandidate sc, ASTNode node) {
-    Set<String> cols = new LinkedHashSet<>();
-    getAllColumnsOfNode(node, cols);
-    return !sc.getColumns().containsAll(cols);
-  }
-
-  /**
    * Set the default value "0.0" in the non answerable aggreagte expressions.
    * @param node
    * @param sc
@@ -467,7 +458,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
     // Iterate over the StorageCandidates and add non projected having columns in inner select ASTs
     for (StorageCandidateHQLContext sc : storageCandidates) {
       aliasDecider.setCounter(selectAliasCounter);
-      processHavingAST(sc.getQueryAst().getSelectAST(), aliasDecider, sc.getStorageCandidate());
+      processHavingAST(sc.getQueryAst().getSelectAST(), aliasDecider, sc);
     }
     removeRedundantProjectedPhrases();
   }
@@ -493,7 +484,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
       ASTNode child = (ASTNode) selectAST.getChild(i);
       ASTNode outerSelect = new ASTNode(child);
       ASTNode selectExprAST = (ASTNode) child.getChild(0);
-      ASTNode outerAST = getOuterAST(selectExprAST, innerSelectAST, aliasDecider, sc.getStorageCandidate(), true,
+      ASTNode outerAST = getOuterAST(selectExprAST, innerSelectAST, aliasDecider, sc, true,
           cubeql.getBaseCube().getDimAttributeNames());
       outerSelect.addChild(outerAST);
       // has an alias? add it
@@ -529,13 +520,14 @@ public class UnionQueryWriter extends SimpleHQLContext {
    5. If given ast is memorized as mentioned in the above cases, return the mapping.
  */
   private ASTNode getOuterAST(ASTNode astNode, ASTNode innerSelectAST,
-      AliasDecider aliasDecider, StorageCandidate sc, boolean isSelectAst, Set<String> dimensionSet)
+      AliasDecider aliasDecider, StorageCandidateHQLContext scContext, boolean isSelectAst, Set<String> dimensionSet)
       throws LensException {
+    StorageCandidate sc = scContext.getStorageCandidate();
     if (astNode == null) {
       return null;
     }
     Set<String> msrCols = new HashSet<>();
-    getAllColumnsOfNode(astNode, msrCols);
+    getAllColumnsOfNode(astNode, msrCols, scContext);
     msrCols.removeAll(dimensionSet);
     if (isAggregateAST(astNode) && sc.getColumns().containsAll(msrCols)) {
       return processAggregate(astNode, innerSelectAST, aliasDecider, isSelectAst);
@@ -544,7 +536,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
       ASTNode exprCopy = MetastoreUtil.copyAST(astNode);
       setDefaultValueInExprForAggregateNodes(exprCopy, sc);
       outerAST.addChild(getOuterAST(getSelectExpr(exprCopy, null, true),
-          innerSelectAST, aliasDecider, sc, isSelectAst, dimensionSet));
+          innerSelectAST, aliasDecider, scContext, isSelectAst, dimensionSet));
       return outerAST;
     } else {
       if (hasAggregate(astNode)) {
@@ -552,10 +544,12 @@ public class UnionQueryWriter extends SimpleHQLContext {
         for (Node child : astNode.getChildren()) {
           ASTNode childAST = (ASTNode) child;
           if (hasAggregate(childAST) && sc.getColumns().containsAll(msrCols)) {
-            outerAST.addChild(getOuterAST(childAST, innerSelectAST, aliasDecider, sc, isSelectAst, dimensionSet));
+            outerAST.addChild(getOuterAST(childAST, innerSelectAST, aliasDecider,
+                scContext, isSelectAst, dimensionSet));
           } else if (hasAggregate(childAST) && !sc.getColumns().containsAll(msrCols)) {
             childAST.replaceChildren(1, 1,  getSelectExpr(null, null, true));
-            outerAST.addChild(getOuterAST(childAST, innerSelectAST, aliasDecider, sc, isSelectAst, dimensionSet));
+            outerAST.addChild(getOuterAST(childAST, innerSelectAST, aliasDecider,
+                scContext, isSelectAst, dimensionSet));
           } else {
             outerAST.addChild(childAST);
           }
@@ -643,7 +637,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
    */
 
   private void processHavingExpression(ASTNode innerSelectAst, Set<ASTNode> havingAggASTs,
-      AliasDecider aliasDecider, StorageCandidate sc) throws LensException {
+      AliasDecider aliasDecider, StorageCandidateHQLContext sc) throws LensException {
     // iterate over all children of the ast and get outer ast corresponding to it.
     for (ASTNode child : havingAggASTs) {
       if (!innerToOuterSelectASTs.containsKey(new HQLParser.HashableASTNode(child))) {
@@ -677,18 +671,51 @@ public class UnionQueryWriter extends SimpleHQLContext {
    * @param msrs
    * @return
    */
-  private Set<String> getAllColumnsOfNode(ASTNode node, Set<String> msrs) {
+  private Set<String> getAllColumnsOfNode(ASTNode node, Set<String> msrs, StorageCandidateHQLContext sc) {
     if (node.getToken().getType() == HiveParser.DOT) {
-      msrs.add(node.getChild(1).toString());
+      String col = node.getChild(1).toString();
+      msrs.addAll(getSourceColumnOfRefColumn(col, sc));
     }
     for (int i = 0; i < node.getChildCount(); i++) {
       ASTNode child = (ASTNode) node.getChild(i);
-      getAllColumnsOfNode(child, msrs);
+      getAllColumnsOfNode(child, msrs, sc);
     }
     return msrs;
   }
 
   /**
+   * Returns the source column of the ref column
+   *
+   * @param refCol
+   * @return
+   */
+  private Set<String> getSourceColumnOfRefColumn(String refCol, StorageCandidateHQLContext sc) {
+    Set<String> sourceColumns = new HashSet<String>();
+    for (Map.Entry<String, Set<String>> entry : sc.getCubeQueryContext().getTblAliasToColumns().entrySet()) {
+      if (entry.getValue().contains(refCol)) {
+        String table = entry.getKey();
+
+        if (sc.getCubeQueryContext().getAutoJoinCtx() != null) {
+          for (Map.Entry<Aliased<Dimension>, List<JoinPath>> dimPaths
+              : sc.getCubeQueryContext().getAutoJoinCtx().getAllPaths().entrySet()) {
+
+            if (dimPaths.getKey().alias.equals(table)) {
+              List<JoinPath> joinPaths = dimPaths.getValue();
+              for (JoinPath path : joinPaths) {
+                sourceColumns.addAll(path.getColumnsForTable(sc.getCubeQueryContext().getBaseCube()));
+              }
+            }
+          }
+        }
+      }
+    }
+    if (sourceColumns.isEmpty()) {
+      sourceColumns.add(refCol);
+    }
+    return sourceColumns;
+  }
+
+  /**
    * Gets from string of the ouer query, this is a union query of all
    * StorageCandidates participated.
    *

http://git-wip-us.apache.org/repos/asf/lens/blob/d49f45a0/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeSegmentationRewriter.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeSegmentationRewriter.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeSegmentationRewriter.java
index 9fa31dc..7e1714b 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeSegmentationRewriter.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeSegmentationRewriter.java
@@ -234,8 +234,7 @@ public class TestCubeSegmentationRewriter extends TestQueryRewrite {
     userQuery = "select cityid, segmsr1 from testcube where cityname='blah' and "
         + TWO_DAYS_RANGE + " having citysegmsr1 > 20";
     String rewrittenQuery = rewrite(userQuery, getConf());
-    assertTrue(rewrittenQuery.toLowerCase().endsWith("sum(case  when ((cubecity.name) = 'foo') "
-        + "then (testcube.segmsr1) end) > 20)"));
+    assertTrue(rewrittenQuery.toLowerCase().endsWith("(sum((testcube.alias2)) > 20)"));
 
     // Order by on alias
     userQuery = "select cityid as `city_id_alias`, segmsr1 from testcube where cityname='blah' and "

http://git-wip-us.apache.org/repos/asf/lens/blob/d49f45a0/lens-cube/src/test/java/org/apache/lens/cube/parse/TestUnionAndJoinCandidates.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestUnionAndJoinCandidates.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestUnionAndJoinCandidates.java
index dc06ead..ccc3bf4 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestUnionAndJoinCandidates.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestUnionAndJoinCandidates.java
@@ -92,6 +92,32 @@ public class TestUnionAndJoinCandidates extends TestQueryRewrite {
   }
 
   @Test
+  public void testExpressionHavingRefcol() throws ParseException, LensException {
+    String colsSelected = " union_join_ctx_cityid, union_join_ctx_cityname_msr1_expr, "
+        + "union_join_ctx_cityname_msr2_expr ";
+    String whereCond = "(" + TWO_MONTHS_RANGE_UPTO_DAYS + ")";
+    String rewrittenQuery = rewrite("select " + colsSelected + " from basecube where " + whereCond, conf);
+    assertTrue(rewrittenQuery.contains("UNION ALL"));
+    String expectedInnerSelect1 = "SELECT (basecube.union_join_ctx_cityid) as `alias0`, sum(case  "
+        + "when ((cubecityjoinunionctx.name) = 'blr') then (basecube.union_join_ctx_msr1) else 0 end) "
+        + "as `alias1`, 0.0 as `alias2` FROM TestQueryRewrite.c1_union_join_ctx_fact1 basecube ";
+    String expectedInnerSelect2 = "SELECT (basecube.union_join_ctx_cityid) as `alias0`, "
+        + "sum(case  when ((cubecityjoinunionctx.name) = 'blr') then (basecube.union_join_ctx_msr1) else 0 end) "
+        + "as `alias1`, 0.0 as `alias2` FROM TestQueryRewrite.c1_union_join_ctx_fact2 basecube";
+    String expectedInnerSelect3 = "SELECT (basecube.union_join_ctx_cityid) as `alias0`, 0.0 as `alias1`, "
+        + "sum(case  when ((cubecityjoinunionctx.name) = 'blr') then (basecube.union_join_ctx_msr2) else 0 end) "
+        + "as `alias2` FROM TestQueryRewrite.c1_union_join_ctx_fact3 basecube";
+    String outerSelect = "SELECT (basecube.alias0) as `union_join_ctx_cityid`, sum((basecube.alias1)) "
+        + "as `union_join_ctx_cityname_msr1_expr`, sum((basecube.alias2)) as `union_join_ctx_cityname_msr2_expr` FROM";
+    String outerGroupBy = "GROUP BY (basecube.alias0)";
+    compareContains(expectedInnerSelect1, rewrittenQuery);
+    compareContains(expectedInnerSelect2, rewrittenQuery);
+    compareContains(expectedInnerSelect3, rewrittenQuery);
+    compareContains(outerSelect, rewrittenQuery);
+    compareContains(outerGroupBy, rewrittenQuery);
+  }
+
+  @Test
   public void testCustomExpressionForJoinCandidate() throws ParseException, LensException {
     // Expr : (case when union_join_ctx_msr2_expr = 0 then 0 else
     // union_join_ctx_msr4_expr * 100 / union_join_ctx_msr2_expr end) is completely answered by

http://git-wip-us.apache.org/repos/asf/lens/blob/d49f45a0/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/cubes/base/basecube.xml b/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
index bcea938..6bb5eb9 100644
--- a/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
+++ b/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
@@ -360,6 +360,17 @@
                 description="union_join_ctx_Not null cityid">
       <expr_spec expr="case when union_join_ctx_cityid is null then 0 else union_join_ctx_cityid end"/>
     </expression>
+
+    <expression _type="int" name="union_join_ctx_cityname_msr1_expr" display_string="union_join_ctx_cityname_msr1_expr"
+                description="union_join_ctx_cityname_msr1_expr">
+      <expr_spec expr="sum(case when union_join_ctx_cityname = 'blr' then union_join_ctx_msr1 else 0 end)"/>
+    </expression>
+
+    <expression _type="int" name="union_join_ctx_cityname_msr2_expr" display_string="union_join_ctx_cityname_msr2_expr"
+                description="union_join_ctx_cityname_msr2_expr">
+      <expr_spec expr="sum(case when union_join_ctx_cityname = 'blr' then union_join_ctx_msr2 else 0 end)"/>
+    </expression>
+
     <expression _type="String" name="cityandstatenew" display_string="City and State"
                 description="city and state together">
       <expr_spec expr="concat(cityname, &quot;:&quot;, statename_cube)" end_time="$gregorian{now.month-2months}"/>

http://git-wip-us.apache.org/repos/asf/lens/blob/d49f45a0/lens-cube/src/test/resources/schema/cubes/base/testcube.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/cubes/base/testcube.xml b/lens-cube/src/test/resources/schema/cubes/base/testcube.xml
index 088c8de..9d2bb02 100644
--- a/lens-cube/src/test/resources/schema/cubes/base/testcube.xml
+++ b/lens-cube/src/test/resources/schema/cubes/base/testcube.xml
@@ -206,6 +206,7 @@
     <expression _type="int" name="notnullcityid" display_string="Not null cityid Expr" description="Not null cityid">
       <expr_spec expr="case when cityid is null then 0 else cityid end"/>
     </expression>
+
     <expression _type="double" name="roundedmsr1" display_string="Rounded msr1" description="rounded measure1">
       <expr_spec expr="round(msr1/1000)"/>
     </expression>


[26/50] lens git commit: LENS-1456: Fix Deadlock in jdbcdriver

Posted by ra...@apache.org.
LENS-1456: Fix Deadlock in jdbcdriver


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

Branch: refs/heads/master
Commit: 6a45685b6baec3db0aaac50065c7c1adf57fdf3f
Parents: cb92ec5
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Fri Aug 11 15:52:48 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Mon Aug 21 13:01:00 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/lens/server/api/query/QueryContext.java  | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/6a45685b/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 ccdef87..466fb25 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
@@ -572,7 +572,7 @@ public class QueryContext extends AbstractQueryContext implements FailureContext
       getDriverStatus().setStatusMessage("Query " + getQueryHandleString() + " " + state.name().toLowerCase());
     }
     getDriverStatus().setState(state);
-    synchronized (this.driverStatusUpdateListeners) {
+    synchronized (this) {
       for (QueryDriverStatusUpdateListener listener : this.driverStatusUpdateListeners) {
         listener.onDriverStatusUpdated(getQueryHandle(), getDriverStatus());
       }
@@ -588,10 +588,8 @@ public class QueryContext extends AbstractQueryContext implements FailureContext
   }
 
 
-  public void registerStatusUpdateListener(QueryDriverStatusUpdateListener driverStatusUpdateListener) {
-    synchronized (this.driverStatusUpdateListeners) {
-      this.driverStatusUpdateListeners.add(driverStatusUpdateListener);
-    }
+  public synchronized void registerStatusUpdateListener(QueryDriverStatusUpdateListener driverStatusUpdateListener) {
+    this.driverStatusUpdateListeners.add(driverStatusUpdateListener);
   }
 
   @Override


[17/50] lens git commit: LENS-1451: Enforcing valid fact tables doesn't filter out segmentations

Posted by ra...@apache.org.
LENS-1451: Enforcing valid fact tables doesn't filter out segmentations


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

Branch: refs/heads/master
Commit: 3769ef0e8987caf26c931bf2735e3658e30f3ac4
Parents: 41051ea
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Thu Jul 6 19:35:15 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:43:00 2017 +0530

----------------------------------------------------------------------
 .../lens/cube/parse/CandidateTableResolver.java | 26 ++++++++++----------
 1 file changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/3769ef0e/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
index be3b474..86209bd 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
@@ -104,6 +104,9 @@ class CandidateTableResolver implements ContextRewriter {
 
   private void populateCandidateTables(CubeQueryContext cubeql) throws LensException {
     if (cubeql.getCube() != null) {
+      String str = cubeql.getConf().get(CubeQueryConfUtil.getValidFactTablesKey(cubeql.getCube().getName()));
+      List<String> validFactTables =
+        StringUtils.isBlank(str) ? null : Arrays.asList(StringUtils.split(str.toLowerCase(), ","));
       List<FactTable> factTables = cubeql.getMetastoreClient().getAllFacts(cubeql.getCube());
       if (factTables.isEmpty()) {
         throw new LensException(LensCubeErrorCode.NO_CANDIDATE_FACT_AVAILABLE.getLensErrorInfo(),
@@ -112,11 +115,15 @@ class CandidateTableResolver implements ContextRewriter {
       for (FactTable fact : factTables) {
         if (fact.getUpdatePeriods().isEmpty()) {
           log.info("Not considering fact: {} as it has no update periods", fact.getName());
+        } else if (validFactTables != null && !validFactTables.contains(fact.getName())) {
+          log.info("Not considering fact: {} as it's not valid as per user configuration.", fact.getName());
         } else {
           for (String s : fact.getStorages()) {
             StorageCandidate sc = new StorageCandidate(cubeql.getCube(), fact, s, cubeql);
             if (isStorageSupportedOnDriver(sc.getStorageName())) {
               cubeql.getCandidates().add(sc);
+            } else {
+              log.info("Not considering {} since storage is not supported on driver.", sc.getName());
             }
           }
         }
@@ -125,7 +132,12 @@ class CandidateTableResolver implements ContextRewriter {
       log.info("Populated storage candidates: {}", cubeql.getCandidates());
       List<SegmentationCandidate> segmentationCandidates = Lists.newArrayList();
       for (Segmentation segmentation : cubeql.getMetastoreClient().getAllSegmentations(cubeql.getCube())) {
-        segmentationCandidates.add(new SegmentationCandidate(cubeql, segmentation));
+        if (validFactTables != null && !validFactTables.contains(segmentation.getName())) {
+          log.info("Not considering segmentation: {} as it's not valid as per user configuration.",
+            segmentation.getName());
+        } else {
+          segmentationCandidates.add(new SegmentationCandidate(cubeql, segmentation));
+        }
       }
       cubeql.getCandidates().addAll(segmentationCandidates);
     }
@@ -261,9 +273,6 @@ class CandidateTableResolver implements ContextRewriter {
 
   private void resolveCandidateFactTables(CubeQueryContext cubeql) throws LensException {
     if (cubeql.getCube() != null) {
-      String str = cubeql.getConf().get(CubeQueryConfUtil.getValidFactTablesKey(cubeql.getCube().getName()));
-      List<String> validFactTables =
-          StringUtils.isBlank(str) ? null : Arrays.asList(StringUtils.split(str.toLowerCase(), ","));
 
       Set<QueriedPhraseContext> queriedMsrs = new HashSet<>();
       Set<QueriedPhraseContext> dimExprs = new HashSet<>();
@@ -279,15 +288,6 @@ class CandidateTableResolver implements ContextRewriter {
         Candidate cand = i.next();
         if (cand instanceof StorageCandidate) {
           StorageCandidate sc = (StorageCandidate) cand;
-          if (validFactTables != null) {
-            if (!validFactTables.contains(sc.getFact().getName().toLowerCase())) {
-              log.info("Not considering storage candidate:{} as it is not a valid candidate", sc);
-              cubeql.addStoragePruningMsg(sc, new CandidateTablePruneCause(CandidateTablePruneCode.INVALID));
-              i.remove();
-              continue;
-            }
-          }
-
           // update expression evaluability for this fact
           for (String expr : cubeql.getQueriedExprs()) {
             cubeql.getExprCtx().updateEvaluables(expr, sc);


[46/50] lens git commit: LENS-1501: Skip cobertura analysis while generating the site

Posted by ra...@apache.org.
LENS-1501: Skip cobertura analysis while generating the site


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

Branch: refs/heads/master
Commit: c2538198bcf4ec58a2df05c64d1fd94adf579e3c
Parents: 0240f6b
Author: rajub <ra...@lazada.com>
Authored: Tue Jan 9 07:53:03 2018 +0800
Committer: rajub <ra...@lazada.com>
Committed: Tue Jan 9 07:53:03 2018 +0800

----------------------------------------------------------------------
 tools/scripts/generate-site-public.sh | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/c2538198/tools/scripts/generate-site-public.sh
----------------------------------------------------------------------
diff --git a/tools/scripts/generate-site-public.sh b/tools/scripts/generate-site-public.sh
index a547465..574caf5 100755
--- a/tools/scripts/generate-site-public.sh
+++ b/tools/scripts/generate-site-public.sh
@@ -44,7 +44,7 @@ CURR_BRANCH=`git branch | sed -n '/\* /s///p'`
 echo "Running site in current lens branch" $CURR_BRANCH
 mvn clean test -Dtest=org.apache.lens.doc.TestGenerateConfigDoc,org.apache.lens.cli.doc.TestGenerateCLIUserDoc -DskipCheck || die "Unable to generate config docs"
 mvn install -DskipTests -DskipCheck
-mvn site site:stage -Ddependency.locations.enabled=false -Ddependency.details.enabled=false -Pjavadoc || die "unable to generate site"
+mvn site site:stage -Ddependency.locations.enabled=false -Ddependency.details.enabled=false -Dcobertura.skip=true -Pjavadoc || die "unable to generate site"
 
 echo "Site gen complete"
 
@@ -92,4 +92,4 @@ rm -r $SITE_BACKUP/site/publish/versions/$VERSION
 cp -r $SITE_BACKUP/site/publish/versions $SVN_TARGET/site/publish/
 cp -r $TMP/ $SVN_TARGET/site/publish
 cd $SVN_TARGET
-echo "Generated site."
\ No newline at end of file
+echo "Generated site."


[47/50] lens git commit: LENS-1500: Fix for Test case failures

Posted by ra...@apache.org.
LENS-1500: Fix for Test case failures


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

Branch: refs/heads/master
Commit: 165c354b6abeb99fe96c176c57a74ed01f0767a6
Parents: c253819
Author: Rajitha R <ra...@gmail.com>
Authored: Tue Jan 23 13:02:21 2018 +0530
Committer: rajub <ra...@lazada.com>
Committed: Wed Jan 24 07:58:53 2018 +0800

----------------------------------------------------------------------
 lens-cube/src/test/resources/schema/facts/testfact.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/165c354b/lens-cube/src/test/resources/schema/facts/testfact.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/testfact.xml b/lens-cube/src/test/resources/schema/facts/testfact.xml
index aa9322a..37041f8 100644
--- a/lens-cube/src/test/resources/schema/facts/testfact.xml
+++ b/lens-cube/src/test/resources/schema/facts/testfact.xml
@@ -46,7 +46,7 @@
     <property name="cube.fact.testfact.c5.updateperiods"
               value="MONTHLY,HOURLY,YEARLY,CONTINUOUS,QUARTERLY,MINUTELY,DAILY"/>
     <property name="cube.fact.testfact.cubename" value="testCube"/>
-    <property name="cube.fact.relative.start.time" value="now.year - 90 days"/>
+    <property name="cube.fact.relative.start.time" value="now.year - 100 days"/>
     <property name="cube.fact.testfact.c4.updateperiods"
               value="MONTHLY,HOURLY,YEARLY,CONTINUOUS,QUARTERLY,MINUTELY,DAILY"/>
     <property name="cube.table.testfact.weight" value="5.0"/>
@@ -248,4 +248,4 @@
       </update_periods>
     </storage_table>
   </storage_tables>
-</x_fact_table>
\ No newline at end of file
+</x_fact_table>


[39/50] lens git commit: LENS-1485: Add javadoc plugin to build plugins section and disable doclint

Posted by ra...@apache.org.
LENS-1485: Add javadoc plugin to build plugins section and disable doclint


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

Branch: refs/heads/master
Commit: 0f08b69b25d430f16e81a99388f0ce83e5e40aa2
Parents: b423d5c
Author: rajub <ra...@lazada.com>
Authored: Wed Nov 29 13:31:55 2017 +0800
Committer: rajub <ra...@lazada.com>
Committed: Wed Nov 29 13:31:55 2017 +0800

----------------------------------------------------------------------
 pom.xml | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/0f08b69b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4b1974b..4aa7854 100644
--- a/pom.xml
+++ b/pom.xml
@@ -420,6 +420,7 @@
           <reportSet>
             <id>aggregate</id>
             <configuration>
+              <additionalparam>-Xdoclint:none</additionalparam>
               <quiet>true</quiet>
               <verbose>false</verbose>
               <includeDependencySources>false</includeDependencySources>
@@ -480,6 +481,14 @@
       <plugins>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-javadoc-plugin</artifactId>
+          <version>${javadoc.plugin.version}</version>
+          <configuration>
+            <additionalparam>-Xdoclint:none</additionalparam>
+          </configuration>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-project-info-reports-plugin</artifactId>
           <version>${reports.plugin.version}</version>
           <configuration>


[04/50] lens git commit: LENS-1433: Virtual Fact over a fact with single storage - multi update period is writing wrong query

Posted by ra...@apache.org.
LENS-1433: Virtual Fact over a fact with single storage - multi update period is writing wrong query


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

Branch: refs/heads/master
Commit: 28aa4a1e25121849e8ada9e0b843d525e2e1f479
Parents: d3875b4
Author: Rajitha R <ra...@gmail.com>
Authored: Wed Jun 14 15:06:56 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Mon Jun 19 09:04:45 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/lens/cube/parse/StorageCandidate.java    | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/28aa4a1e/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
index c8ff3b8..6e5aa4c 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
@@ -265,7 +265,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
       Set<String> uniqueStorageTables = new HashSet<>();
       for (UpdatePeriod updatePeriod : validUpdatePeriods) {
         uniqueStorageTables.add(
-          getCubeMetastoreClient().getStorageTableName(fact.getName(), storageName, updatePeriod)
+          getCubeMetastoreClient().getStorageTableName(fact.getSourceFactName(), storageName, updatePeriod)
         );
       }
       return uniqueStorageTables;
@@ -891,7 +891,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
       return this.startTime;
     }
     return getCubeMetastoreClient().getStorageTableStartDate(
-      getCubeMetastoreClient().getStorageTableName(fact.getName(), storageName, interval), fact.getName());
+      getCubeMetastoreClient().getStorageTableName(fact.getSourceFactName(), storageName, interval), fact.getName());
   }
 
   private Date getStorageTableEndDate(UpdatePeriod interval) throws LensException {
@@ -900,7 +900,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
       return this.endTime;
     }
     return getCubeMetastoreClient().getStorageTableEndDate(
-      getCubeMetastoreClient().getStorageTableName(fact.getName(), storageName, interval), fact.getName());
+      getCubeMetastoreClient().getStorageTableName(fact.getSourceFactName(), storageName, interval), fact.getName());
   }
 
 
@@ -931,7 +931,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
       StorageCandidate updatePeriodSpecificSc;
       for (UpdatePeriod period : participatingUpdatePeriods) {
         updatePeriodSpecificSc = copy();
-        updatePeriodSpecificSc.setResolvedName(getCubeMetastoreClient().getStorageTableName(fact.getName(),
+        updatePeriodSpecificSc.setResolvedName(getCubeMetastoreClient().getStorageTableName(fact.getSourceFactName(),
           storageName, period));
         updatePeriodSpecificSc.truncatePartitions(period);
         periodSpecificScList.add(updatePeriodSpecificSc);


[50/50] lens git commit: Reverse merge current-release-line

Posted by ra...@apache.org.
Reverse merge current-release-line


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

Branch: refs/heads/master
Commit: 9e7012ed9c4eb163ce7285042a10dcad935f9522
Parents: d0b78e8 865d202
Author: rajub <ra...@lazada.com>
Authored: Tue Feb 6 13:43:51 2018 +0800
Committer: rajub <ra...@lazada.com>
Committed: Tue Feb 6 13:43:51 2018 +0800

----------------------------------------------------------------------
 README.md                                         |  4 ++--
 contrib/clients/python/pom.xml                    |  3 +--
 .../apache/lens/cube/parse/TestCubeRewriter.java  | 13 +++++++++++++
 .../lens/server/query/TestQueryService.java       |  2 +-
 .../lens/server/scheduler/SchedulerDAOTest.java   |  2 +-
 lens-ui/app/actions/SessionAction.js              | 18 ++++++++++++++++++
 lens-ui/app/adapters/SessionAdapter.js            | 18 ++++++++++++++++++
 lens-ui/app/components/SessionListComponent.js    | 18 ++++++++++++++++++
 lens-ui/app/constants/SessionConstants.js         | 18 ++++++++++++++++++
 lens-ui/app/stores/SessionStore.js                | 18 ++++++++++++++++++
 pom.xml                                           | 12 ++++++++++--
 src/site/apt/releases/download.apt                |  2 +-
 src/site/apt/releases/release-history.apt         | 10 ++++++++++
 tools/scripts/generate-site-public.sh             |  2 +-
 14 files changed, 130 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/9e7012ed/contrib/clients/python/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lens/blob/9e7012ed/lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java
----------------------------------------------------------------------
diff --cc lens-cube/src/test/java/org/apache/lens/cube/parse/TestCubeRewriter.java
index e83eacb,9b54957..4fdb822
--- 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
@@@ -1467,58 -1467,15 +1467,71 @@@ public class TestCubeRewriter extends T
          .getParticipatingPartitions());
      // pt does not exist beyond 1 day. So in this test, max look ahead possible is 3
      assertEquals(partsQueried, expectedPartsQueried);
 +
 +    //New test case for testing look ahead time parts
 +    expectedPartsQueried = Sets.newTreeSet();
 +    conf.setInt(CubeQueryConfUtil.LOOK_AHEAD_TIME_PARTS_PFX + "daily", 2);
 +
 +    ceilDay = DAILY.getCeilDate(getDateWithOffset(DAILY, -3));
 +    nextDay = DateUtils.addDays(ceilDay, 1);
 +    nextToNextDay = DateUtils.addDays(nextDay, 1);
 +
 +    for (TimePartition p : Iterables.concat(
 +      TimePartition.of(HOURLY, getDateWithOffset(DAILY, -3)).rangeUpto(TimePartition.of(HOURLY, ceilDay)),
 +      TimePartition.of(DAILY, ceilDay).rangeUpto(TimePartition.of(DAILY, nextDay)),
 +      TimePartition.of(DAILY, nextDay).rangeUpto(TimePartition.of(DAILY, nextToNextDay)),
 +      TimePartition.of(HOURLY, nextToNextDay).rangeUpto(TimePartition.of(HOURLY, NOW)))) {
 +      FactPartition fp = new FactPartition("it", p, null, storageTables);
 +      expectedPartsQueried.add(fp);
 +    }
 +    for (TimePartition it : TimePartition.of(HOURLY, ceilDay).rangeUpto(TimePartition.of(HOURLY, nextDay))) {
 +      for (TimePartition pt : TimePartition.of(HOURLY, nextDay).rangeUpto(TimePartition.of(HOURLY, nextToNextDay))) {
 +        FactPartition ptPartition = new FactPartition("pt", pt, null, storageTables);
 +        FactPartition itPartition = new FactPartition("it", it, ptPartition, storageTables);
 +        expectedPartsQueried.add(itPartition);
 +      }
 +    }
 +
 +    ceilDay = DateUtils.addDays(ceilDay, 1);
 +    nextDay = DateUtils.addDays(nextDay, 1);
 +    nextToNextDay = DateUtils.addDays(nextToNextDay, 1);
 +
 +    for (TimePartition it : TimePartition.of(HOURLY, ceilDay).rangeUpto(TimePartition.of(HOURLY, nextDay))) {
 +      for (TimePartition pt : TimePartition.of(HOURLY, nextDay).rangeUpto(TimePartition.of(HOURLY, nextToNextDay))) {
 +        FactPartition ptPartition = new FactPartition("pt", pt, null, storageTables);
 +        FactPartition itPartition = new FactPartition("it", it, ptPartition, storageTables);
 +        expectedPartsQueried.add(itPartition);
 +      }
 +    }
 +
 +    ctx = rewriteCtx("select dim1, max(msr3)," + " msr2 from testCube" + " where " + THREE_DAYS_RANGE_IT,
 +      conf);
 +    partsQueried = new TreeSet<>(((StorageCandidate)ctx.getCandidates().iterator().next())
 +      .getParticipatingPartitions());
 +    // pt does not exist beyond 1 day. So in this test, max look ahead possible is 3
 +    assertEquals(partsQueried, expectedPartsQueried);
 +  }
 +
 +  @Test
 +  public void testTimeRangeIn() throws Exception {
 +    //check whether time_range_in is resolving in cube rewrite
 +    Configuration conf = getConf();
 +    conf.set(CubeQueryConfUtil.PROCESS_TIME_PART_COL, "pt");
 +    conf.set(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "true");
 +    conf.setClass(CubeQueryConfUtil.TIME_RANGE_WRITER_CLASS, AbridgedTimeRangeWriter.class, TimeRangeWriter.class);
++    CubeQueryContext ctx = rewriteCtx("select dim1, sum(msr23)" + " from testCube" + " where " + ONE_DAY_RANGE_IT,
++      conf);
++    String rewrittenQuery = ctx.toHQL();
++    assertTrue(!rewrittenQuery.contains("time_range_in"));
+   }
+ 
+   @Test
+   public void testTimeRangeIn() throws Exception {
+     //check whether time_range_in is resolving in cube rewrite
+     Configuration conf = getConf();
+     conf.set(CubeQueryConfUtil.PROCESS_TIME_PART_COL, "pt");
+     conf.set(CubeQueryConfUtil.FAIL_QUERY_ON_PARTIAL_DATA, "true");
+     conf.setClass(CubeQueryConfUtil.TIME_RANGE_WRITER_CLASS, AbridgedTimeRangeWriter.class, TimeRangeWriter.class);
      CubeQueryContext ctx = rewriteCtx("select dim1, sum(msr23)" + " from testCube" + " where " + ONE_DAY_RANGE_IT,
        conf);
      String rewrittenQuery = ctx.toHQL();

http://git-wip-us.apache.org/repos/asf/lens/blob/9e7012ed/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lens/blob/9e7012ed/tools/scripts/generate-site-public.sh
----------------------------------------------------------------------


[03/50] lens git commit: LENS-1434: Segmentation Candidate should have dynamic cost depending on facts picked in segmented cubes

Posted by ra...@apache.org.
LENS-1434: Segmentation Candidate should have dynamic cost depending on facts picked in segmented cubes


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

Branch: refs/heads/master
Commit: d3875b4e9f46b97114819ea3318d76c1b25cc62b
Parents: 13ee285
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Tue Jun 6 18:25:05 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Sat Jun 10 13:30:24 2017 +0800

----------------------------------------------------------------------
 .../org/apache/lens/cube/parse/Candidate.java   |  2 +-
 .../lens/cube/parse/CubeQueryRewriter.java      | 12 +++----
 .../apache/lens/cube/parse/JoinCandidate.java   | 12 +++++--
 .../lens/cube/parse/LightestFactResolver.java   | 36 +++++++++++---------
 .../lens/cube/parse/SegmentationCandidate.java  | 19 +++++++++--
 .../lens/cube/parse/StorageCandidate.java       | 22 +++++++-----
 .../cube/parse/StorageCandidateHQLContext.java  |  9 +++++
 .../apache/lens/cube/parse/UnionCandidate.java  | 15 +++++---
 .../lens/cube/parse/UnionQueryWriter.java       | 17 +++++----
 9 files changed, 95 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/d3875b4e/lens-cube/src/main/java/org/apache/lens/cube/parse/Candidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/Candidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/Candidate.java
index 0855ced..ffa8fb6 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/Candidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/Candidate.java
@@ -105,7 +105,7 @@ public interface Candidate {
   /**
    * @return the cost of this candidate
    */
-  double getCost();
+  OptionalDouble getCost();
 
   /**
    * Returns true if this candidate contains the given candidate

http://git-wip-us.apache.org/repos/asf/lens/blob/d3875b4e/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
index d064cdb..0ef41f3 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
@@ -149,6 +149,7 @@ public class CubeQueryRewriter {
     DenormalizationResolver denormResolver = new DenormalizationResolver();
     CandidateTableResolver candidateTblResolver = new CandidateTableResolver();
     StorageTableResolver storageTableResolver = new StorageTableResolver(conf);
+    LightestFactResolver lightestFactResolver = new LightestFactResolver();
 
     // Phase 1 of exprResolver: Resolve expressions
     rewriters.add(exprResolver);
@@ -185,7 +186,7 @@ public class CubeQueryRewriter {
       rewriters.add(exprResolver);
       // Pick the least cost combination(s) (and prune others) out of a set of combinations produced
       // by CandidateCoveringSetsResolver
-      rewriters.add(new LightestFactResolver());
+      rewriters.add(lightestFactResolver);
     }
 
     // Phase 2 of storageTableResolver: resolve storage table partitions.
@@ -206,11 +207,10 @@ public class CubeQueryRewriter {
     // Phase 2 of exprResolver : Prune candidate facts without any valid expressions
     rewriters.add(exprResolver);
 
-    if (!lightFactFirst) {
-      // Pick the least cost combination(s) (and prune others) out of a set of combinations produced
-      // by CandidateCoveringSetsResolver
-      rewriters.add(new LightestFactResolver());
-    }
+    // Pick the least cost combination(s) (and prune others) out of a set of combinations produced
+    // by CandidateCoveringSetsResolver
+    rewriters.add(lightestFactResolver);
+
     // if two combinations have the same least weight/cost, then the combination with least number of time partitions
     // queried will be picked. Rest of the combinations will be pruned
     rewriters.add(new LeastPartitionResolver());

http://git-wip-us.apache.org/repos/asf/lens/blob/d3875b4e/lens-cube/src/main/java/org/apache/lens/cube/parse/JoinCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/JoinCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/JoinCandidate.java
index c4049cd..d9915f4 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/JoinCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/JoinCandidate.java
@@ -72,8 +72,16 @@ public class JoinCandidate implements Candidate {
   }
 
   @Override
-  public double getCost() {
-    return children.stream().mapToDouble(Candidate::getCost).sum();
+  public OptionalDouble getCost() {
+    double cost = 0;
+    for (Candidate candidate : getChildren()) {
+      if (candidate.getCost().isPresent()) {
+        cost += candidate.getCost().getAsDouble();
+      } else {
+        return OptionalDouble.empty();
+      }
+    }
+    return OptionalDouble.of(cost);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lens/blob/d3875b4e/lens-cube/src/main/java/org/apache/lens/cube/parse/LightestFactResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/LightestFactResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/LightestFactResolver.java
index dd25f3e..52e3632 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/LightestFactResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/LightestFactResolver.java
@@ -19,7 +19,11 @@
 
 package org.apache.lens.cube.parse;
 
-import java.util.*;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
 
 import org.apache.lens.cube.parse.CandidateTablePruneCause.CandidateTablePruneCode;
 import org.apache.lens.server.api.error.LensException;
@@ -35,21 +39,21 @@ public class LightestFactResolver implements ContextRewriter {
   @Override
   public void rewriteContext(CubeQueryContext cubeql) throws LensException {
     if (cubeql.getCube() != null && !cubeql.getCandidates().isEmpty()) {
-      Map<Candidate, Double> factWeightMap = new HashMap<Candidate, Double>();
-
-      for (Candidate cand : cubeql.getCandidates()) {
-        factWeightMap.put(cand, cand.getCost());
-      }
-
-      double minWeight = Collections.min(factWeightMap.values());
-
-      for (Iterator<Candidate> i = cubeql.getCandidates().iterator(); i.hasNext();) {
-        Candidate cand = i.next();
-        if (factWeightMap.get(cand) > minWeight) {
-          log.info("Not considering candidate:{} from final candidates as it has more fact weight:{} minimum:{}",
-            cand, factWeightMap.get(cand), minWeight);
-          cubeql.addCandidatePruningMsg(cand, new CandidateTablePruneCause(CandidateTablePruneCode.MORE_WEIGHT));
-          i.remove();
+      Map<Candidate, Double> factWeightMap = cubeql.getCandidates().stream()
+        .filter(candidate -> candidate.getCost().isPresent())
+        .collect(Collectors.toMap(Function.identity(), x -> x.getCost().getAsDouble()));
+      if (!factWeightMap.isEmpty()) {
+        double minWeight = Collections.min(factWeightMap.values());
+        for (Iterator<Candidate> i = cubeql.getCandidates().iterator(); i.hasNext();) {
+          Candidate cand = i.next();
+          if (factWeightMap.containsKey(cand)) {
+            if (factWeightMap.get(cand) > minWeight) {
+              log.info("Not considering candidate:{} from final candidates as it has more fact weight:{} minimum:{}",
+                cand, factWeightMap.get(cand), minWeight);
+              cubeql.addCandidatePruningMsg(cand, new CandidateTablePruneCause(CandidateTablePruneCode.MORE_WEIGHT));
+              i.remove();
+            }
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/lens/blob/d3875b4e/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
index a359d86..a2bd485 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
@@ -37,6 +37,7 @@ import java.util.Date;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
+import java.util.OptionalDouble;
 import java.util.Set;
 import java.util.function.Predicate;
 import java.util.stream.Collector;
@@ -238,8 +239,20 @@ public class SegmentationCandidate implements Candidate {
   }
 
   @Override
-  public double getCost() {
-    return segmentation.weight();
+  public OptionalDouble getCost() {
+    if (areCandidatesPicked()) {
+      double cost = 0.0;
+      for (Candidate candidate : getChildren()) {
+        if (candidate.getCost().isPresent()) {
+          cost += candidate.getCost().getAsDouble();
+        } else {
+          return OptionalDouble.empty();
+        }
+      }
+      return OptionalDouble.of(cost);
+    } else {
+      return OptionalDouble.empty();
+    }
   }
 
   @Override
@@ -281,7 +294,7 @@ public class SegmentationCandidate implements Candidate {
     // I can't ask my children to check this context for evaluability.
     return cubeStream()
       .map(cube -> cube.getExpressionByName(expr.getExprCol().getName()))
-      .allMatch(Predicate.isEqual(expr.getExprCol()));
+      .allMatch(Objects::nonNull);
   }
 
   private boolean areCandidatesPicked() {

http://git-wip-us.apache.org/repos/asf/lens/blob/d3875b4e/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
index 7980797..c8ff3b8 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
@@ -41,6 +41,7 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.OptionalDouble;
 import java.util.Set;
 import java.util.TimeZone;
 import java.util.TreeSet;
@@ -298,6 +299,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
     return (AbstractCubeTable) cube;
   }
 
+  @Override
   public StorageCandidate copy() throws LensException {
     return new StorageCandidate(this);
   }
@@ -307,6 +309,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
     return phrase.isEvaluable(this);
   }
 
+  @Override
   public AbstractCubeTable getTable() {
     return (AbstractCubeTable) fact;
   }
@@ -350,8 +353,8 @@ public class StorageCandidate implements Candidate, CandidateTable {
   }
 
   @Override
-  public double getCost() {
-    return fact.weight();
+  public OptionalDouble getCost() {
+    return OptionalDouble.of(fact.weight());
   }
 
   @Override
@@ -367,7 +370,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
   private void updatePartitionStorage(FactPartition part) throws LensException {
     try {
       if (getCubeMetastoreClient().factPartitionExists(fact, part, storageTable)) {
-        part.getStorageTables().add(name);
+        part.getStorageTables().add(storageTable);
         part.setFound(true);
       }
     } catch (HiveException e) {
@@ -417,10 +420,10 @@ public class StorageCandidate implements Candidate, CandidateTable {
       && cubeQueryContext.getRangeWriter().getClass().equals(BetweenTimeRangeWriter.class)) {
       FactPartition part = new FactPartition(partCol, fromDate, maxInterval, null, partWhereClauseFormat);
       partitions.add(part);
-      part.getStorageTables().add(storageName);
+      part.getStorageTables().add(storageTable);
       part = new FactPartition(partCol, toDate, maxInterval, null, partWhereClauseFormat);
       partitions.add(part);
-      part.getStorageTables().add(storageName);
+      part.getStorageTables().add(storageTable);
       this.participatingUpdatePeriods.add(maxInterval);
       log.info("Added continuous fact partition for storage table {}", storageName);
       return true;
@@ -534,7 +537,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
             missingPartitions.add(part);
             if (!failOnPartialData) {
               partitions.add(part);
-              part.getStorageTables().add(storageName);
+              part.getStorageTables().add(storageTable);
             }
           } else {
             log.info("No finer granualar partitions exist for {}", part);
@@ -735,7 +738,6 @@ public class StorageCandidate implements Candidate, CandidateTable {
 
   @Override
   public boolean isDimAttributeEvaluable(String dim) throws LensException {
-
     return getCubeQueryContext().getDeNormCtx()
       .addRefUsage(getCubeQueryContext(), this, dim, getCubeQueryContext().getCube().getName());
   }
@@ -929,9 +931,9 @@ public class StorageCandidate implements Candidate, CandidateTable {
       StorageCandidate updatePeriodSpecificSc;
       for (UpdatePeriod period : participatingUpdatePeriods) {
         updatePeriodSpecificSc = copy();
-        updatePeriodSpecificSc.truncatePartitions(period);
         updatePeriodSpecificSc.setResolvedName(getCubeMetastoreClient().getStorageTableName(fact.getName(),
           storageName, period));
+        updatePeriodSpecificSc.truncatePartitions(period);
         periodSpecificScList.add(updatePeriodSpecificSc);
       }
       periodSpecificStorageCandidates = periodSpecificScList;
@@ -949,6 +951,10 @@ public class StorageCandidate implements Candidate, CandidateTable {
     while (rangeItr.hasNext()) {
       Map.Entry<TimeRange, Set<FactPartition>> rangeEntry = rangeItr.next();
       rangeEntry.getValue().removeIf(factPartition -> !factPartition.getPeriod().equals(updatePeriod));
+      rangeEntry.getValue().forEach(factPartition -> {
+        factPartition.getStorageTables().remove(storageTable);
+        factPartition.getStorageTables().add(resolvedName);
+      });
       if (rangeEntry.getValue().isEmpty()) {
         rangeItr.remove();
       }

http://git-wip-us.apache.org/repos/asf/lens/blob/d3875b4e/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
index c535196..cca39c0 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
@@ -150,4 +150,13 @@ public class StorageCandidateHQLContext extends DimHQLContext {
       }
     }
   }
+
+  @Override
+  public int hashCode() {
+    final int PRIME = 59;
+    int result = 1;
+    result = result * PRIME + getStorageCandidate().hashCode();
+    result = result * PRIME + getCube().hashCode();
+    return result;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/d3875b4e/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionCandidate.java
index 757a877..510ea0c 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionCandidate.java
@@ -30,6 +30,7 @@ import java.util.List;
 import java.util.ListIterator;
 import java.util.Map;
 import java.util.Optional;
+import java.util.OptionalDouble;
 import java.util.Set;
 
 import org.apache.lens.cube.metadata.FactPartition;
@@ -154,14 +155,18 @@ public class UnionCandidate implements Candidate {
   }
 
   @Override
-  public double getCost() {
+  public OptionalDouble getCost() {
     double cost = 0.0;
     for (TimeRange timeRange : getCubeQueryContext().getTimeRanges()) {
       for (Map.Entry<Candidate, TimeRange> entry : getTimeRangeSplit(timeRange).entrySet()) {
-        cost += entry.getKey().getCost() * entry.getValue().milliseconds() / timeRange.milliseconds();
+        if (entry.getKey().getCost().isPresent()) {
+          cost +=  entry.getKey().getCost().getAsDouble() *entry.getValue().milliseconds() / timeRange.milliseconds();
+        } else {
+          return OptionalDouble.empty();
+        }
       }
     }
-    return cost;
+    return OptionalDouble.of(cost);
   }
 
   @Override
@@ -256,7 +261,9 @@ public class UnionCandidate implements Candidate {
    * @return
    */
   private Map<Candidate, TimeRange> splitTimeRangeForChildren(TimeRange timeRange) {
-    children.sort(comparing(Candidate::getCost));
+    if (children.stream().map(Candidate::getCost).allMatch(OptionalDouble::isPresent)) {
+      children.sort(comparing(x -> x.getCost().getAsDouble()));
+    }
     Map<Candidate, TimeRange> childrenTimeRangeMap = new HashMap<>();
     // Sorted list based on the weights.
     Set<TimeRange> ranges = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/lens/blob/d3875b4e/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
index 9412f27..cc0a2e5 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
@@ -48,7 +48,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
 
   private Map<HQLParser.HashableASTNode, ASTNode> innerToOuterSelectASTs = new HashMap<>();
   private Map<HQLParser.HashableASTNode, ASTNode> innerToOuterHavingASTs = new HashMap<>();
-  private Map<String, ASTNode> storageCandidateToSelectAstMap = new HashMap<>();
+  private Map<StorageCandidateHQLContext, ASTNode> storageCandidateToSelectAstMap = new HashMap<>();
   private CubeQueryContext cubeql;
   static final ASTNode DEFAULT_MEASURE_AST;
   private static final String DEFAULT_MEASURE = "0.0";
@@ -91,8 +91,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
    */
   private void updateAsts() {
     for (StorageCandidateHQLContext sc : storageCandidates) {
-      storageCandidateToSelectAstMap.put(sc.getStorageCandidate().toString(),
-          new ASTNode(new CommonToken(TOK_SELECT, "TOK_SELECT")));
+      storageCandidateToSelectAstMap.put(sc, new ASTNode(new CommonToken(TOK_SELECT, "TOK_SELECT")));
       if (sc.getQueryAst().getHavingAST() != null) {
         cubeql.setHavingAST(sc.getQueryAst().getHavingAST());
       }
@@ -403,7 +402,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
       if (!phrase.hasMeasures(cubeql)) {
         for (StorageCandidateHQLContext sc : storageCandidates) {
           ASTNode exprWithOutAlias = (ASTNode) sc.getQueryAst().getSelectAST().getChild(i).getChild(0);
-          storageCandidateToSelectAstMap.get(sc.getStorageCandidate().toString()).
+          storageCandidateToSelectAstMap.get(sc).
               addChild(getSelectExpr(exprWithOutAlias, aliasNode, false));
         }
 
@@ -412,7 +411,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
         for (StorageCandidateHQLContext sc : storageCandidates) {
           if (sc.getStorageCandidate().getAnswerableMeasurePhraseIndices().contains(phrase.getPosition())) {
             ASTNode exprWithOutAlias = (ASTNode) sc.getQueryAst().getSelectAST().getChild(i).getChild(0);
-            storageCandidateToSelectAstMap.get(sc.getStorageCandidate().toString()).
+            storageCandidateToSelectAstMap.get(sc).
                 addChild(getSelectExpr(exprWithOutAlias, aliasNode, false));
           } else {
             ASTNode resolvedExprNode = getAggregateNodesExpression(i);
@@ -421,7 +420,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
             } else {
               resolvedExprNode = getSelectExpr(null, null, true);
             }
-            storageCandidateToSelectAstMap.get(sc.getStorageCandidate().toString()).
+            storageCandidateToSelectAstMap.get(sc).
                 addChild(getSelectExpr(resolvedExprNode, aliasNode, false));
           }
         }
@@ -431,7 +430,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
         for (StorageCandidateHQLContext sc : storageCandidates) {
           if (sc.getStorageCandidate().getAnswerableMeasurePhraseIndices().contains(phrase.getPosition())) {
             ASTNode exprWithOutAlias = (ASTNode) sc.getQueryAst().getSelectAST().getChild(i).getChild(0);
-            storageCandidateToSelectAstMap.get(sc.getStorageCandidate().toString()).
+            storageCandidateToSelectAstMap.get(sc).
                 addChild(getSelectExpr(exprWithOutAlias, aliasNode, false));
           } else {
             ASTNode resolvedExprNode = getAggregateNodesExpression(i);
@@ -440,7 +439,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
             } else {
               resolvedExprNode = getSelectExpr(null, null, true);
             }
-            storageCandidateToSelectAstMap.get(sc.getStorageCandidate().toString()).
+            storageCandidateToSelectAstMap.get(sc).
                 addChild(getSelectExpr(resolvedExprNode, aliasNode, false));
           }
         }
@@ -485,7 +484,7 @@ public class UnionQueryWriter extends SimpleHQLContext {
   private void processSelectExpression(StorageCandidateHQLContext sc, ASTNode outerSelectAst, ASTNode innerSelectAST,
       AliasDecider aliasDecider) throws LensException {
     //ASTNode selectAST = sc.getQueryAst().getSelectAST();
-    ASTNode selectAST = storageCandidateToSelectAstMap.get(sc.getStorageCandidate().toString());
+    ASTNode selectAST = storageCandidateToSelectAstMap.get(sc);
     if (selectAST == null) {
       return;
     }


[18/50] lens git commit: LENS-1452: Optimize Time Union candidate Algorithm

Posted by ra...@apache.org.
LENS-1452: Optimize Time Union candidate Algorithm


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

Branch: refs/heads/master
Commit: 6dca44661bf604ca1436c6cd1d3998405d0333a4
Parents: 3769ef0
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Mon Jul 10 16:52:54 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:43:03 2017 +0530

----------------------------------------------------------------------
 .../parse/CandidateCoveringSetsResolver.java    | 47 +++++++++++++++++---
 1 file changed, 41 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/6dca4466/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
index 8e07162..69d9562 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
@@ -26,6 +26,7 @@ import org.apache.lens.cube.error.LensCubeErrorCode;
 import org.apache.lens.cube.metadata.TimeRange;
 import org.apache.lens.server.api.error.LensException;
 
+import com.google.common.collect.Lists;
 import lombok.extern.slf4j.Slf4j;
 
 @Slf4j
@@ -124,16 +125,14 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
       }
     }
     // Get all covering fact sets
-    List<UnionCandidate> unionCoveringSet =
-        getCombinations(new ArrayList<>(allCandidatesPartiallyValid), cubeql);
+//    List<UnionCandidate> unionCoveringSet = getCombinations(new ArrayList<>(allCandidatesPartiallyValid), cubeql);
+    List<UnionCandidate> unionCoveringSet = getCombinationTailIterative(allCandidatesPartiallyValid, cubeql);
     // Sort the Collection based on no of elements
     unionCoveringSet.sort(Comparator.comparing(Candidate::getChildrenCount));
     // prune candidate set which doesn't contain any common measure i
     if (!queriedMsrs.isEmpty()) {
       pruneUnionCoveringSetWithoutAnyCommonMeasure(unionCoveringSet, queriedMsrs);
     }
-    // prune redundant covering sets
-    pruneRedundantUnionCoveringSets(unionCoveringSet);
     // pruing done in the previous steps, now create union candidates
     candidateSet.addAll(unionCoveringSet);
     updateQueriableMeasures(candidateSet, qpcList);
@@ -155,7 +154,7 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
       }
     }
   }
-
+  @Deprecated
   private void pruneRedundantUnionCoveringSets(List<UnionCandidate> candidates) {
     for (int i = 0; i < candidates.size(); i++) {
       UnionCandidate current = candidates.get(i);
@@ -168,7 +167,7 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
       }
     }
   }
-
+  @Deprecated
   private List<UnionCandidate> getCombinations(final List<Candidate> candidates, CubeQueryContext cubeql) {
     List<UnionCandidate> combinations = new LinkedList<>();
     int size = candidates.size();
@@ -193,6 +192,42 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
     return combinations;
   }
 
+  /**
+   * The following function is iterative rewrite of the following tail-recursive implementation:
+   * (ignoring cubeql for clarity)
+   * getCombinations(candidates) = getCombinationsTailRecursive(emptyList(), candidates)
+   *
+   * getCombinationsTailRecursive(incompleteCombinations: List[List[Candidate]], candidates: List[Candidate]) =
+   *   head, tail = head and tail of linked List candidates
+   *   add head to all elements of incompleteCombinations.
+   *   complete = remove now complete combinations from incompleteCombinations
+   *   return complete ++ getCombinationsTailRecursive(incompleteCombinations, tail)
+   * @param candidates
+   * @param cubeql
+   * @return
+   */
+  private List<UnionCandidate> getCombinationTailIterative(List<Candidate> candidates, CubeQueryContext cubeql) {
+    LinkedList<Candidate> candidateLinkedList = Lists.newLinkedList(candidates);
+    List<List<Candidate>> incompleteCombinations = Lists.newArrayList();
+    List<UnionCandidate> unionCandidates = Lists.newArrayList();
+
+    while(!candidateLinkedList.isEmpty()) {
+      Candidate candidate = candidateLinkedList.remove();
+      incompleteCombinations.add(Lists.newArrayList());
+      Iterator<List<Candidate>> iter = incompleteCombinations.iterator();
+      while(iter.hasNext()) {
+        List<Candidate> incompleteCombination = iter.next();
+        incompleteCombination.add(candidate);
+        UnionCandidate unionCandidate = new UnionCandidate(incompleteCombination, cubeql);
+        if (isCandidateCoveringTimeRanges(unionCandidate, cubeql.getTimeRanges())) {
+          unionCandidates.add(unionCandidate);
+          iter.remove();
+        }
+      }
+    }
+    return unionCandidates;
+  }
+
   private List<List<Candidate>> resolveJoinCandidates(List<Candidate> candidates,
     Set<QueriedPhraseContext> msrs) throws LensException {
     List<List<Candidate>> msrCoveringSets = new ArrayList<>();


[10/50] lens git commit: LENS-1444: Optimize the algorithm of finding all eligible union candidates

Posted by ra...@apache.org.
LENS-1444: Optimize the algorithm of finding all eligible union candidates


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

Branch: refs/heads/master
Commit: a7f407bcb59ffa84c8ab6e830ba98aee81516085
Parents: c174583
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Fri Jun 23 16:39:37 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:42:52 2017 +0530

----------------------------------------------------------------------
 .../cube/parse/CandidateCoveringSetsResolver.java  | 17 ++++-------------
 1 file changed, 4 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/a7f407bc/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
index 61c28c6..8e07162 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
@@ -106,16 +106,6 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
     return true;
   }
 
-  private void pruneUnionCandidatesNotCoveringAllRanges(List<UnionCandidate> ucs, CubeQueryContext cubeql) {
-    for (Iterator<UnionCandidate> itr = ucs.iterator(); itr.hasNext();) {
-      UnionCandidate uc = itr.next();
-      if (!isCandidateCoveringTimeRanges(uc, cubeql.getTimeRanges())) {
-        itr.remove();
-        cubeql.addCandidatePruningMsg(uc, CandidateTablePruneCause.storageNotAvailableInRange(cubeql.getTimeRanges()));
-      }
-    }
-  }
-
   private List<Candidate> resolveTimeRangeCoveringFactSet(CubeQueryContext cubeql,
       Set<QueriedPhraseContext> queriedMsrs, List<QueriedPhraseContext> qpcList) throws LensException {
     List<Candidate> candidateSet = new ArrayList<>();
@@ -138,8 +128,6 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
         getCombinations(new ArrayList<>(allCandidatesPartiallyValid), cubeql);
     // Sort the Collection based on no of elements
     unionCoveringSet.sort(Comparator.comparing(Candidate::getChildrenCount));
-    // prune non covering sets
-    pruneUnionCandidatesNotCoveringAllRanges(unionCoveringSet, cubeql);
     // prune candidate set which doesn't contain any common measure i
     if (!queriedMsrs.isEmpty()) {
       pruneUnionCoveringSetWithoutAnyCommonMeasure(unionCoveringSet, queriedMsrs);
@@ -197,7 +185,10 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
         clonedI = clonedI >>> 1;
         --count;
       }
-      combinations.add(new UnionCandidate(individualCombinationList, cubeql));
+      UnionCandidate uc = new UnionCandidate(individualCombinationList, cubeql);
+      if (isCandidateCoveringTimeRanges(uc, cubeql.getTimeRanges())) {
+        combinations.add(uc);
+      }
     }
     return combinations;
   }


[06/50] lens git commit: LENS-1439: Having clause getting skipped if query goes to a segmentation containing single segment

Posted by ra...@apache.org.
LENS-1439: Having clause getting skipped if query goes to a segmentation containing single segment


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

Branch: refs/heads/master
Commit: 051412bec7eea93ef8b4dfad4e10c47e10754dfa
Parents: 42d60d2
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Fri Jun 16 14:53:41 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Mon Jun 19 09:05:36 2017 +0800

----------------------------------------------------------------------
 .../org/apache/lens/cube/parse/StorageCandidateHQLContext.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/051412be/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
index cca39c0..494b08e 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
@@ -50,7 +50,7 @@ public class StorageCandidateHQLContext extends DimHQLContext {
     this.storageCandidate = storageCandidate;
     this.rootCubeQueryContext = rootCubeQueryContext;
     getCubeQueryContext().addRangeClauses(this);
-    if (!isRoot()) {
+    if (!Objects.equals(getStorageCandidate(), rootCubeQueryContext.getPickedCandidate())) {
       getQueryAst().setHavingAST(null);
     }
   }


[02/50] lens git commit: LENS-1432 : Map Join tuning broken for this query leading to Map timeouts

Posted by ra...@apache.org.
LENS-1432 : Map Join tuning broken for this query leading to Map timeouts


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

Branch: refs/heads/master
Commit: 13ee28519fbb8aa92c30abdff4a286155fddeef5
Parents: 419e190
Author: Sushil Mohanty <su...@gmail.com>
Authored: Tue Jun 6 18:14:39 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Sat Jun 10 13:30:08 2017 +0800

----------------------------------------------------------------------
 .../cube/parse/MultiCandidateQueryWriterContext.java     | 11 +++++++++++
 .../org/apache/lens/cube/parse/QueryWriterContext.java   |  4 ++++
 .../java/org/apache/lens/driver/cube/RewriterPlan.java   |  6 +++---
 .../org/apache/lens/cube/parse/TestRewriterPlan.java     |  2 ++
 .../lens/cube/parse/TestUnionAndJoinCandidates.java      |  2 +-
 lens-cube/src/test/resources/schema/facts/b1fact1.xml    |  2 +-
 6 files changed, 22 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/13ee2851/lens-cube/src/main/java/org/apache/lens/cube/parse/MultiCandidateQueryWriterContext.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/MultiCandidateQueryWriterContext.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/MultiCandidateQueryWriterContext.java
index d57c027..50046a5 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/MultiCandidateQueryWriterContext.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/MultiCandidateQueryWriterContext.java
@@ -18,8 +18,11 @@
  */
 package org.apache.lens.cube.parse;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
+import org.apache.lens.cube.metadata.Dimension;
 import org.apache.lens.server.api.error.LensException;
 
 import com.google.common.collect.Lists;
@@ -90,4 +93,12 @@ public class MultiCandidateQueryWriterContext implements QueryWriterContext {
     List<StorageCandidateHQLContext> leafWriterContexts = getLeafQueryWriterContexts();
     return new UnionQueryWriter(leafWriterContexts, getCubeQueryContext());
   }
+
+  public Map<Dimension, CandidateDim> getDimsToQuery() {
+    Map<Dimension, CandidateDim> allDimsQueried = new HashMap<>();
+    for (QueryWriterContext ctx : children) {
+      allDimsQueried.putAll(ctx.getDimsToQuery());
+    }
+    return allDimsQueried;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/13ee2851/lens-cube/src/main/java/org/apache/lens/cube/parse/QueryWriterContext.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/QueryWriterContext.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/QueryWriterContext.java
index d55de1f..6531f22 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/QueryWriterContext.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/QueryWriterContext.java
@@ -18,6 +18,9 @@
  */
 package org.apache.lens.cube.parse;
 
+import java.util.Map;
+
+import org.apache.lens.cube.metadata.Dimension;
 import org.apache.lens.server.api.error.LensException;
 
 /**
@@ -31,4 +34,5 @@ public interface QueryWriterContext {
   QueryAST getQueryAst();
   void updateFromString() throws LensException;
   QueryWriter toQueryWriter() throws LensException;
+  Map<Dimension, CandidateDim> getDimsToQuery();
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/13ee2851/lens-cube/src/main/java/org/apache/lens/driver/cube/RewriterPlan.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/driver/cube/RewriterPlan.java b/lens-cube/src/main/java/org/apache/lens/driver/cube/RewriterPlan.java
index caf8770..a27aec1 100644
--- a/lens-cube/src/main/java/org/apache/lens/driver/cube/RewriterPlan.java
+++ b/lens-cube/src/main/java/org/apache/lens/driver/cube/RewriterPlan.java
@@ -45,10 +45,10 @@ public final class RewriterPlan extends DriverQueryPlan {
 
   @SuppressWarnings("unchecked") // required for (Set<FactPartition>) casting
   void extractPlan(Collection<CubeQueryContext> cubeQueries) {
-
     for (CubeQueryContext ctx : cubeQueries) {
-      if (ctx.getPickedDimTables() != null && !ctx.getPickedDimTables().isEmpty()) {
-        for (CandidateDim dim : ctx.getPickedDimTables()) {
+      if (ctx.getQueryWriterContext().getDimsToQuery() != null
+          && !ctx.getQueryWriterContext().getDimsToQuery().isEmpty()) {
+        for (CandidateDim dim : ctx.getQueryWriterContext().getDimsToQuery().values()) {
           addTablesQueried(dim.getStorageTable());
           if (partitions.get(dim.getName()) == null || partitions.get(dim.getName()).isEmpty()) {
             // puts storage table to latest part

http://git-wip-us.apache.org/repos/asf/lens/blob/13ee2851/lens-cube/src/test/java/org/apache/lens/cube/parse/TestRewriterPlan.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestRewriterPlan.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestRewriterPlan.java
index 76ea77d..6223df7 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestRewriterPlan.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestRewriterPlan.java
@@ -71,6 +71,8 @@ public class TestRewriterPlan extends TestQueryRewrite {
       + " cubecity.name != \"XYZ\" and " + TWO_DAYS_RANGE + " having sum(msr2) > 1000 order by cubecity.name limit 50",
       conf);
     ctx.toHQL();
+    // One dimension table queried
+    Assert.assertEquals(ctx.getQueryWriterContext().getDimsToQuery().size(), 1);
     RewriterPlan plan = new RewriterPlan(Collections.singleton(ctx));
     Assert.assertNotNull(plan);
     Assert.assertFalse(plan.getTablesQueried().isEmpty());

http://git-wip-us.apache.org/repos/asf/lens/blob/13ee2851/lens-cube/src/test/java/org/apache/lens/cube/parse/TestUnionAndJoinCandidates.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestUnionAndJoinCandidates.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestUnionAndJoinCandidates.java
index 429e1c6..dc06ead 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/parse/TestUnionAndJoinCandidates.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/TestUnionAndJoinCandidates.java
@@ -147,7 +147,7 @@ public class TestUnionAndJoinCandidates extends TestQueryRewrite {
     compareContains(outerGroupBy, rewrittenQuery);
   }
 
-  @Test(invocationCount = 100)
+  @Test
   public void testFinalCandidateRewrittenQuery() throws ParseException, LensException {
     try {
       // Query with non projected measure in having clause.

http://git-wip-us.apache.org/repos/asf/lens/blob/13ee2851/lens-cube/src/test/resources/schema/facts/b1fact1.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/facts/b1fact1.xml b/lens-cube/src/test/resources/schema/facts/b1fact1.xml
index b51a34a..1500037 100644
--- a/lens-cube/src/test/resources/schema/facts/b1fact1.xml
+++ b/lens-cube/src/test/resources/schema/facts/b1fact1.xml
@@ -19,7 +19,7 @@
   under the License.
 
 -->
-<x_fact_table name="b1fact1" cube_name="b1cube" weight="5.0" xmlns="uri:lens:cube:0.1">
+<x_fact_table name="b1fact1" cube_name="b1cube" weight="5.1" xmlns="uri:lens:cube:0.1">
   <columns>
     <column name="zipcode" _type="int" comment="zip"/>
     <column name="cityid" _type="int" comment="city id"/>


[25/50] lens git commit: LENS-1458 : RESULT_SET_PARENT_DIR set at driver level getting overwritten for queued queries after server restart

Posted by ra...@apache.org.
LENS-1458 : RESULT_SET_PARENT_DIR set at driver level getting overwritten for queued queries after server restart


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

Branch: refs/heads/master
Commit: cb92ec5cdafdee653465c7b6038850992123136b
Parents: 05c364e
Author: Sushil Mohanty <su...@gmail.com>
Authored: Mon Aug 7 18:38:15 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Mon Aug 21 13:00:55 2017 +0530

----------------------------------------------------------------------
 .../apache/lens/server/api/query/DriverSelectorQueryContext.java   | 2 +-
 .../src/test/java/org/apache/lens/server/TestServerRestart.java    | 2 ++
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/cb92ec5c/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java b/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
index 41e533e..9cec42a 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/query/DriverSelectorQueryContext.java
@@ -254,7 +254,7 @@ public class DriverSelectorQueryContext {
   }
 
   public void setDriverConf(LensDriver driver, Configuration conf) {
-    driverQueryContextMap.get(driver).setDriverSpecificConf(conf);
+    driverQueryContextMap.get(driver).setDriverSpecificConf(mergeConf(driver, conf));
   }
 
   public void setDriverCost(LensDriver driver, QueryCost cost) {

http://git-wip-us.apache.org/repos/asf/lens/blob/cb92ec5c/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 20aa8cb..8b7728e 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
@@ -39,6 +39,7 @@ import org.apache.lens.api.APIResult.Status;
 import org.apache.lens.api.query.*;
 import org.apache.lens.api.result.LensAPIResult;
 import org.apache.lens.driver.hive.TestRemoteHiveDriver;
+import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.query.QueryContext;
 import org.apache.lens.server.api.query.QueryExecutionService;
@@ -245,6 +246,7 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
     assertEquals(lensQueryConf.getProperty(KEY_POST_SELECT), VALUE_POST_SELECT);
 
     if (afterRestart) {
+      assertEquals(driverConf.get(LensConfConstants.RESULT_SET_PARENT_DIR), "target/hive-lens-results");
       //This will be unavailable since if was not updated in LensConf by MockDriverQueryHook
       assertNull(driverConf.get(UNSAVED_KEY_POST_SELECT));
     } else {


[31/50] lens git commit: LENS-1474 : Fix NPE in ExpressionResolver for SegmentationCandidate

Posted by ra...@apache.org.
LENS-1474 : Fix NPE in ExpressionResolver for SegmentationCandidate


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

Branch: refs/heads/master
Commit: ae6bda33ed501e15dceeecea83c86adf189e4453
Parents: 717214c
Author: Rajitha R <ra...@gmail.com>
Authored: Mon Sep 11 09:53:21 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Thu Oct 5 11:12:15 2017 +0800

----------------------------------------------------------------------
 .../apache/lens/cube/parse/ExpressionResolver.java | 17 +++++++++++------
 1 file changed, 11 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/ae6bda33/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
index 8906fae..822e25e 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
@@ -314,12 +314,14 @@ class ExpressionResolver implements ContextRewriter {
     }
 
     ExpressionContext getExpressionContext(String expr, String alias) {
-      for (ExpressionContext ec : allExprsQueried.get(expr)) {
-        if (ec.getSrcAlias().equals(alias)) {
-          return ec;
+      if (allExprsQueried.get(expr) != null) {
+        for (ExpressionContext ec : allExprsQueried.get(expr)) {
+          if (ec.getSrcAlias().equals(alias)) {
+            return ec;
+          }
         }
       }
-      throw new IllegalArgumentException("no expression available for " + expr + " alias:" + alias);
+      return null;
     }
 
     boolean hasMeasures(String expr, CubeInterface cube) {
@@ -511,8 +513,11 @@ class ExpressionResolver implements ContextRewriter {
       expressionContexts.add(expressionContext);
       for (StorageCandidate sc : scSet) {
         storageTableNames.add(sc.getStorageTable());
-        expressionContexts.add(sc.getCubeQueryContext().getExprCtx()
-          .getExpressionContext(expressionContext.getExprCol().getName(), expressionContext.getSrcAlias()));
+        ExpressionContext ex = sc.getCubeQueryContext().getExprCtx().getExpressionContext(expressionContext.
+            getExprCol().getName(), expressionContext.getSrcAlias());
+        if (ex != null) {
+          expressionContexts.add(ex);
+        }
       }
       for (ExpressionContext ec : expressionContexts) {
         for (CandidateTable table : ec.evaluableExpressions.keySet()) {


[07/50] lens git commit: LENS-1441: CandidateTableResolver should not add StorageCandidate if storage is not supported

Posted by ra...@apache.org.
LENS-1441: CandidateTableResolver should not add StorageCandidate if storage is not supported


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

Branch: refs/heads/master
Commit: c2a9c9316cc5d3a06b02a157722fa58f2757d47a
Parents: 051412b
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Tue Jun 20 16:08:00 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:42:48 2017 +0530

----------------------------------------------------------------------
 .../lens/cube/parse/CandidateTableResolver.java | 24 +++++++++++++++++++-
 .../lens/cube/parse/CubeQueryRewriter.java      |  2 +-
 .../lens/cube/parse/ExpressionResolver.java     |  5 +++-
 .../lens/cube/parse/StorageTableResolver.java   |  6 -----
 .../cube/parse/TestDenormalizationResolver.java | 10 ++------
 .../lens/cube/parse/TestTimeRangeResolver.java  | 20 +++++++---------
 .../server/query/QueryAPIErrorResponseTest.java |  1 +
 7 files changed, 39 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/c2a9c931/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
index f530650..be3b474 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateTableResolver.java
@@ -29,6 +29,7 @@ import org.apache.lens.cube.parse.ExpressionResolver.ExpressionContext;
 import org.apache.lens.server.api.error.LensException;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
@@ -48,8 +49,15 @@ import lombok.extern.slf4j.Slf4j;
 @Slf4j
 class CandidateTableResolver implements ContextRewriter {
 
+  private final List<String> supportedStorages;
+  private final boolean allStoragesSupported;
   private boolean checkForQueriedColumns = true;
 
+  public CandidateTableResolver(Configuration conf) {
+    this.supportedStorages = getSupportedStorages(conf);
+    this.allStoragesSupported = (supportedStorages == null);
+  }
+
   @Override
   public void rewriteContext(CubeQueryContext cubeql) throws LensException {
     if (checkForQueriedColumns) {
@@ -81,6 +89,18 @@ class CandidateTableResolver implements ContextRewriter {
       checkForQueriedColumns = true;
     }
   }
+  private List<String> getSupportedStorages(Configuration conf) {
+    String[] storages = conf.getStrings(CubeQueryConfUtil.DRIVER_SUPPORTED_STORAGES);
+    if (storages != null) {
+      return Arrays.asList(storages);
+    }
+    return null;
+  }
+
+  private boolean isStorageSupportedOnDriver(String storage) {
+    return allStoragesSupported || supportedStorages.contains(storage);
+  }
+
 
   private void populateCandidateTables(CubeQueryContext cubeql) throws LensException {
     if (cubeql.getCube() != null) {
@@ -95,7 +115,9 @@ class CandidateTableResolver implements ContextRewriter {
         } else {
           for (String s : fact.getStorages()) {
             StorageCandidate sc = new StorageCandidate(cubeql.getCube(), fact, s, cubeql);
-            cubeql.getCandidates().add(sc);
+            if (isStorageSupportedOnDriver(sc.getStorageName())) {
+              cubeql.getCandidates().add(sc);
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/lens/blob/c2a9c931/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
index 0ef41f3..143b266 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CubeQueryRewriter.java
@@ -147,7 +147,7 @@ public class CubeQueryRewriter {
     rewriters.add(new AliasReplacer());
     ExpressionResolver exprResolver = new ExpressionResolver();
     DenormalizationResolver denormResolver = new DenormalizationResolver();
-    CandidateTableResolver candidateTblResolver = new CandidateTableResolver();
+    CandidateTableResolver candidateTblResolver = new CandidateTableResolver(conf);
     StorageTableResolver storageTableResolver = new StorageTableResolver(conf);
     LightestFactResolver lightestFactResolver = new LightestFactResolver();
 

http://git-wip-us.apache.org/repos/asf/lens/blob/c2a9c931/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
index b1654d1..2403576 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
@@ -556,7 +556,10 @@ class ExpressionResolver implements ContextRewriter {
               log.info("Removing expression {} as all tables have non reachable fields", esc);
               iterator.remove();
               removedEsc.add(esc);
-              break;
+              removed = true;
+            }
+            if (removed) {
+              continue;
             }
             //remove expressions which are not valid in the timerange queried
             // If an expression is defined as

http://git-wip-us.apache.org/repos/asf/lens/blob/c2a9c931/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageTableResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageTableResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageTableResolver.java
index 3acd754..d7da8cb 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageTableResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageTableResolver.java
@@ -252,12 +252,6 @@ class StorageTableResolver implements ContextRewriter {
       if (c instanceof StorageCandidate) {
         StorageCandidate sc = (StorageCandidate) c;
         // first check: if the storage is supported on driver
-        if (!isStorageSupportedOnDriver(sc.getStorageName())) {
-          log.info("Skipping storage: {} as it is not supported", sc.getStorageName());
-          cubeql.addStoragePruningMsg(sc, new CandidateTablePruneCause(CandidateTablePruneCode.UNSUPPORTED_STORAGE));
-          it.remove();
-          continue;
-        }
         String str = conf.get(CubeQueryConfUtil.getValidStorageTablesKey(sc.getFact().getName()));
         List<String> validFactStorageTables =
           StringUtils.isBlank(str) ? null : Arrays.asList(StringUtils.split(str.toLowerCase(), ","));

http://git-wip-us.apache.org/repos/asf/lens/blob/c2a9c931/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 ffd0dec..9b217ae 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
@@ -200,16 +200,10 @@ public class TestDenormalizationResolver extends TestQueryRewrite {
     Map<Set<String>, List<CandidateTablePruneCause>> expected = Maps.newHashMap();
     expected.put(newHashSet("c1_summary1", "c1_testfact", "c1_testfact2"),
       newArrayList(columnNotFound("dim2big2")));
-    expected.put(newHashSet("c2_summary2", "c2_summary3", "c1_testfact2_raw", ""
-        + "c3_testfact2_raw", "c1_summary3", "c1_summary2"),
+    expected.put(newHashSet("c1_testfact2_raw", "c1_summary3", "c1_summary2"),
       newArrayList(new CandidateTablePruneCause(CandidateTablePruneCode.INVALID_DENORM_TABLE)));
-    expected.put(newHashSet("c0_b1b2fact1", "c0_testfact_continuous", "SEG[b1cube; b2cube]"),
+    expected.put(newHashSet("SEG[b1cube; b2cube]"),
       newArrayList(columnNotFound("msr2", "msr3")));
-    expected.put(newHashSet("c2_summary2", "c2_summary3", "c2_summary4", "c4_testfact", "c2_summary1",
-      "c3_testfact", "c3_testfact2_raw", "c6_testfact", "c4_testfact2", "c5_testfact", "c99_cheapfact",
-      "c2_testfact", "c0_cheapfact", "c2_testfactmonthly", "c0_testfact"),
-      newArrayList(new CandidateTablePruneCause(CandidateTablePruneCode.UNSUPPORTED_STORAGE)));
-
     Assert.assertEquals(enhanced, expected);
   }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/c2a9c931/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 929fb46..181608f 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
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -21,9 +21,9 @@ package org.apache.lens.cube.parse;
 
 import static org.apache.lens.cube.metadata.DateFactory.*;
 import static org.apache.lens.cube.parse.CandidateTablePruneCause.CandidateTablePruneCode.TIME_RANGE_NOT_ANSWERABLE;
-import static org.apache.lens.cube.parse.CandidateTablePruneCause.CandidateTablePruneCode.UNSUPPORTED_STORAGE;
 
 import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
 
 import java.util.*;
 
@@ -41,8 +41,6 @@ import org.testng.annotations.Test;
 
 public class TestTimeRangeResolver extends TestQueryRewrite {
 
-  private final String cubeName = CubeTestSetup.TEST_CUBE_NAME;
-
   private Configuration conf;
 
   @BeforeTest
@@ -62,7 +60,7 @@ public class TestTimeRangeResolver extends TestQueryRewrite {
 
   @Test
   public void testFactValidity() throws ParseException, LensException, HiveException, ClassNotFoundException {
-    String query = "select msr2 from " + cubeName + " where "  + LAST_YEAR_RANGE;
+    String query = "select msr2 from " + CubeTestSetup.TEST_CUBE_NAME + " where "  + LAST_YEAR_RANGE;
     LensException e = getLensExceptionInRewrite(query, getConf());
     assertEquals(e.getErrorInfo().getErrorName(), "NO_UNION_CANDIDATE_AVAILABLE");
   }
@@ -74,13 +72,11 @@ public class TestTimeRangeResolver extends TestQueryRewrite {
         getConf());
     List<CandidateTablePruneCause> causes = findPruningMessagesForStorage("c3_testfact_deprecated",
       ctx.getStoragePruningMsgs());
-    assertEquals(causes.size(), 1);
-    assertEquals(causes.get(0).getCause(), UNSUPPORTED_STORAGE);
+    assertTrue(causes.isEmpty());
 
     causes = findPruningMessagesForStorage("c4_testfact_deprecated",
       ctx.getStoragePruningMsgs());
-    assertEquals(causes.size(), 1);
-    assertEquals(causes.get(0).getCause(), UNSUPPORTED_STORAGE);
+    assertTrue(causes.isEmpty());
 
     // testfact_deprecated's validity should be in between of both ranges. So both ranges should be in the invalid list
     // That would prove that parsing of properties has gone through successfully
@@ -109,8 +105,8 @@ public class TestTimeRangeResolver extends TestQueryRewrite {
   /**
    *
    * @param stoargeName  storageName_factName
-   * @param allStoragePruningMsgs
-   * @return
+   * @param allStoragePruningMsgs all pruning messages
+   * @return pruning messages for storagetable
    */
   private static List<CandidateTablePruneCause> findPruningMessagesForStorage(String stoargeName,
     PruneCauses<Candidate> allStoragePruningMsgs) {
@@ -121,6 +117,6 @@ public class TestTimeRangeResolver extends TestQueryRewrite {
         }
       }
     }
-    return  new ArrayList<CandidateTablePruneCause>();
+    return new ArrayList<>();
   }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/c2a9c931/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 5409d21..82bb505 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
@@ -284,6 +284,7 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
       //Create a StorageTable
       XStorageTables tables = new XStorageTables();
       tables.getStorageTable().add(createStorageTblElement(testStorage, "DAILY"));
+      tables.getStorageTable().add(createStorageTblElement("mydb", "DAILY")); // for jdbc
       xFactTable.setStorageTables(tables);
 
       createFactFailFast(target, sessionId, xFactTable, mt);


[42/50] lens git commit: [maven-release-plugin] prepare release apache-lens-2.7.0

Posted by ra...@apache.org.
[maven-release-plugin] prepare release apache-lens-2.7.0


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

Branch: refs/heads/master
Commit: d2b49929dd2af6a401a8ea6fa2d357c270a6890e
Parents: 8f9b568
Author: rajub <ra...@lazada.com>
Authored: Tue Dec 5 05:45:01 2017 +0800
Committer: rajub <ra...@lazada.com>
Committed: Tue Dec 5 05:45:01 2017 +0800

----------------------------------------------------------------------
 checkstyle/pom.xml             |  6 +++++-
 contrib/clients/pom.xml        |  2 +-
 contrib/clients/python/pom.xml |  5 ++---
 contrib/pom.xml                |  2 +-
 lens-api/pom.xml               |  2 +-
 lens-cli/pom.xml               |  2 +-
 lens-client/pom.xml            |  2 +-
 lens-cube/pom.xml              |  2 +-
 lens-dist/pom.xml              |  2 +-
 lens-driver-es/pom.xml         |  2 +-
 lens-driver-hive/pom.xml       |  2 +-
 lens-driver-jdbc/pom.xml       |  2 +-
 lens-examples/pom.xml          |  2 +-
 lens-ml-dist/pom.xml           |  2 +-
 lens-ml-lib/pom.xml            |  2 +-
 lens-query-lib/pom.xml         |  2 +-
 lens-regression/pom.xml        |  2 +-
 lens-server-api/pom.xml        |  2 +-
 lens-server/pom.xml            |  2 +-
 lens-ship-jars/pom.xml         |  2 +-
 lens-storage-db/pom.xml        |  2 +-
 lens-ui/pom.xml                |  2 +-
 pom.xml                        | 11 +++++------
 23 files changed, 32 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/checkstyle/pom.xml
----------------------------------------------------------------------
diff --git a/checkstyle/pom.xml b/checkstyle/pom.xml
index a2dc2b3..db378e2 100644
--- a/checkstyle/pom.xml
+++ b/checkstyle/pom.xml
@@ -29,7 +29,7 @@
   <groupId>org.apache.lens</groupId>
   <artifactId>checkstyle</artifactId>
   <name>Lens Checkstyle Rules</name>
-  <version>2.7.0-SNAPSHOT</version>
+  <version>2.7.0</version>
 
   <build>
     <plugins>
@@ -49,4 +49,8 @@
       </plugin>
     </plugins>
   </build>
+
+  <scm>
+    <tag>apache-lens-2.7.0</tag>
+  </scm>
 </project>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/contrib/clients/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/clients/pom.xml b/contrib/clients/pom.xml
index 0e9cd2d..4eeba9a 100644
--- a/contrib/clients/pom.xml
+++ b/contrib/clients/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>lens-contrib</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-contrib-clients</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/contrib/clients/python/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/clients/python/pom.xml b/contrib/clients/python/pom.xml
index ed7b99e..f3969cb 100644
--- a/contrib/clients/python/pom.xml
+++ b/contrib/clients/python/pom.xml
@@ -19,14 +19,13 @@
   under the License.
 
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <name>Lens Python Client</name>
   <parent>
     <artifactId>lens-contrib-clients</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-python-client</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/contrib/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/pom.xml b/contrib/pom.xml
index 35dde79..7ce7982 100644
--- a/contrib/pom.xml
+++ b/contrib/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-contrib</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-api/pom.xml
----------------------------------------------------------------------
diff --git a/lens-api/pom.xml b/lens-api/pom.xml
index d37c471..ab3cd25 100644
--- a/lens-api/pom.xml
+++ b/lens-api/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-api</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-cli/pom.xml
----------------------------------------------------------------------
diff --git a/lens-cli/pom.xml b/lens-cli/pom.xml
index 8e5e3eb..e0e0b53 100644
--- a/lens-cli/pom.xml
+++ b/lens-cli/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-client/pom.xml
----------------------------------------------------------------------
diff --git a/lens-client/pom.xml b/lens-client/pom.xml
index cf71250..bc85772 100644
--- a/lens-client/pom.xml
+++ b/lens-client/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-cube/pom.xml
----------------------------------------------------------------------
diff --git a/lens-cube/pom.xml b/lens-cube/pom.xml
index e338568..855daba 100644
--- a/lens-cube/pom.xml
+++ b/lens-cube/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-cube</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-dist/pom.xml
----------------------------------------------------------------------
diff --git a/lens-dist/pom.xml b/lens-dist/pom.xml
index 7ef4df8..e82a5a1 100644
--- a/lens-dist/pom.xml
+++ b/lens-dist/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-dist</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-driver-es/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-es/pom.xml b/lens-driver-es/pom.xml
index ee6ae9b..f64846a 100644
--- a/lens-driver-es/pom.xml
+++ b/lens-driver-es/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache.lens</groupId>
     <artifactId>apache-lens</artifactId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-driver-es</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-driver-hive/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-hive/pom.xml b/lens-driver-hive/pom.xml
index c5ef153..ef95d7a 100644
--- a/lens-driver-hive/pom.xml
+++ b/lens-driver-hive/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-driver-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-jdbc/pom.xml b/lens-driver-jdbc/pom.xml
index 93dd4c2..c68d0c6 100644
--- a/lens-driver-jdbc/pom.xml
+++ b/lens-driver-jdbc/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-driver-jdbc</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-examples/pom.xml
----------------------------------------------------------------------
diff --git a/lens-examples/pom.xml b/lens-examples/pom.xml
index ecdf60c..25807a8 100644
--- a/lens-examples/pom.xml
+++ b/lens-examples/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-ml-dist/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ml-dist/pom.xml b/lens-ml-dist/pom.xml
index 10e1877..b737028 100644
--- a/lens-ml-dist/pom.xml
+++ b/lens-ml-dist/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
     
   <artifactId>lens-ml-dist</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-ml-lib/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ml-lib/pom.xml b/lens-ml-lib/pom.xml
index 71f7aaf..e2a809e 100644
--- a/lens-ml-lib/pom.xml
+++ b/lens-ml-lib/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-query-lib/pom.xml
----------------------------------------------------------------------
diff --git a/lens-query-lib/pom.xml b/lens-query-lib/pom.xml
index 4fababe..017cb98 100644
--- a/lens-query-lib/pom.xml
+++ b/lens-query-lib/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-query-lib</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-regression/pom.xml
----------------------------------------------------------------------
diff --git a/lens-regression/pom.xml b/lens-regression/pom.xml
index 13dd552..fbcb35b 100644
--- a/lens-regression/pom.xml
+++ b/lens-regression/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-regression</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-server-api/pom.xml
----------------------------------------------------------------------
diff --git a/lens-server-api/pom.xml b/lens-server-api/pom.xml
index e6943dc..f3750eb 100644
--- a/lens-server-api/pom.xml
+++ b/lens-server-api/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-server-api</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-server/pom.xml
----------------------------------------------------------------------
diff --git a/lens-server/pom.xml b/lens-server/pom.xml
index f56f7bc..807e70e 100644
--- a/lens-server/pom.xml
+++ b/lens-server/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-ship-jars/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ship-jars/pom.xml b/lens-ship-jars/pom.xml
index f5c7d6e..a3c37cf 100644
--- a/lens-ship-jars/pom.xml
+++ b/lens-ship-jars/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-ship-jars</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-storage-db/pom.xml
----------------------------------------------------------------------
diff --git a/lens-storage-db/pom.xml b/lens-storage-db/pom.xml
index 476d065..1b3f9bd 100644
--- a/lens-storage-db/pom.xml
+++ b/lens-storage-db/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-storage-db</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/lens-ui/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ui/pom.xml b/lens-ui/pom.xml
index 9c8bfca..88a2974 100644
--- a/lens-ui/pom.xml
+++ b/lens-ui/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0-SNAPSHOT</version>
+    <version>2.7.0</version>
   </parent>
 
   <artifactId>lens-ui</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/d2b49929/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4aa7854..244beaa 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,8 +19,7 @@
   under the License.
 
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
 
   <parent>
@@ -128,7 +127,7 @@
 
   <groupId>org.apache.lens</groupId>
   <artifactId>apache-lens</artifactId>
-  <version>2.7.0-SNAPSHOT</version>
+  <version>2.7.0</version>
   <name>Lens</name>
   <packaging>pom</packaging>
   <description>Unified Analytics Platform</description>
@@ -379,7 +378,7 @@
     <connection>scm:git:https://git-wip-us.apache.org/repos/asf/lens.git</connection>
     <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/lens.git</developerConnection>
     <url>https://git-wip-us.apache.org/repos/asf/lens.git</url>
-    <tag>HEAD</tag>
+    <tag>apache-lens-2.7.0</tag>
   </scm>
 
   <reporting>
@@ -733,7 +732,7 @@
           <dependency>
             <groupId>org.apache.lens</groupId>
             <artifactId>checkstyle</artifactId>
-            <version>2.7.0-SNAPSHOT</version>
+            <version>2.7.0</version>
           </dependency>
         </dependencies>
         <executions>
@@ -1847,7 +1846,7 @@
               <dependency>
                 <groupId>org.apache.lens</groupId>
                 <artifactId>checkstyle</artifactId>
-                <version>2.7.0-SNAPSHOT</version>
+                <version>2.7.0</version>
               </dependency>
             </dependencies>
             <executions>


[33/50] lens git commit: LENS-1464: One or two queued queries failing with ConcurrentModificationException on restart

Posted by ra...@apache.org.
LENS-1464: One or two queued queries failing with ConcurrentModificationException on restart


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

Branch: refs/heads/master
Commit: 1d6679754240f2980e9bffd278a3ee04ac98db02
Parents: 0f58445
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Wed Aug 30 15:59:23 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Thu Oct 5 11:12:56 2017 +0800

----------------------------------------------------------------------
 lens-server/pom.xml                             |  4 +--
 .../server/query/QueryExecutionServiceImpl.java | 32 +++++++++-----------
 2 files changed, 16 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/1d667975/lens-server/pom.xml
----------------------------------------------------------------------
diff --git a/lens-server/pom.xml b/lens-server/pom.xml
index 2065f04..f56f7bc 100644
--- a/lens-server/pom.xml
+++ b/lens-server/pom.xml
@@ -402,8 +402,8 @@
           </execution>
         </executions>
         <configuration>
-          <source>1.7</source>
-          <target>1.7</target>
+          <source>1.8</source>
+          <target>1.8</target>
           <configFile>enunciate.xml</configFile>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/lens/blob/1d667975/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 e70d290..4d141e5 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
@@ -3476,27 +3476,23 @@ public class QueryExecutionServiceImpl extends BaseLensService implements QueryE
 
     // Add resources if either they haven't been marked as added on the session, or if Hive driver says they need
     // to be added to the corresponding hive driver
-    if (!hiveDriver.areDBResourcesAddedForSession(sessionIdentifier, ctx.getDatabase())) {
-      Collection<ResourceEntry> dbResources = session.getDBResources(ctx.getDatabase());
-
-      if (CollectionUtils.isNotEmpty(dbResources)) {
-        log.info("Proceeding to add resources for DB {} for query {} resources: {}", session.getCurrentDatabase(),
-          ctx.getLogHandle(), dbResources);
-
-        List<ResourceEntry> failedDBResources = addResources(dbResources, sessionHandle, hiveDriver);
-        Iterator<ResourceEntry> itr = dbResources.iterator();
-        while (itr.hasNext()) {
-          ResourceEntry res = itr.next();
-          if (!failedDBResources.contains(res)) {
-            itr.remove();
-          }
+    synchronized (session) {
+      if (!hiveDriver.areDBResourcesAddedForSession(sessionIdentifier, ctx.getDatabase())) {
+        Collection<ResourceEntry> dbResources = session.getDBResources(ctx.getDatabase());
+
+        if (CollectionUtils.isNotEmpty(dbResources)) {
+          log.info("Proceeding to add resources for DB {} for query {} resources: {}", session.getCurrentDatabase(),
+            ctx.getLogHandle(), dbResources);
+
+          List<ResourceEntry> failedDBResources = addResources(dbResources, sessionHandle, hiveDriver);
+          dbResources.removeIf(res -> !failedDBResources.contains(res));
+        } else {
+          log.info("No need to add DB resources for session: {} db= {}", sessionIdentifier,
+            session.getCurrentDatabase());
         }
-      } else {
-        log.info("No need to add DB resources for session: {} db= {}", sessionIdentifier, session.getCurrentDatabase());
+        hiveDriver.setResourcesAddedForSession(sessionIdentifier, ctx.getDatabase());
       }
-      hiveDriver.setResourcesAddedForSession(sessionIdentifier, ctx.getDatabase());
     }
-
     // Get pending session resources which needed to be added for this database
     Collection<ResourceEntry> pendingResources =
       session.getPendingSessionResourcesForDatabase(ctx.getDatabase());


[49/50] lens git commit: [maven-release-plugin] prepare for next development iteration

Posted by ra...@apache.org.
[maven-release-plugin] prepare for next development iteration


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

Branch: refs/heads/master
Commit: 865d20267e8128134bb981f1db9dab9108661061
Parents: 81147a5
Author: rajub <ra...@lazada.com>
Authored: Wed Jan 24 09:17:13 2018 +0800
Committer: rajub <ra...@lazada.com>
Committed: Wed Jan 24 09:17:13 2018 +0800

----------------------------------------------------------------------
 checkstyle/pom.xml             | 6 +-----
 contrib/clients/pom.xml        | 2 +-
 contrib/clients/python/pom.xml | 2 +-
 contrib/pom.xml                | 2 +-
 lens-api/pom.xml               | 2 +-
 lens-cli/pom.xml               | 2 +-
 lens-client/pom.xml            | 2 +-
 lens-cube/pom.xml              | 2 +-
 lens-dist/pom.xml              | 2 +-
 lens-driver-es/pom.xml         | 2 +-
 lens-driver-hive/pom.xml       | 2 +-
 lens-driver-jdbc/pom.xml       | 2 +-
 lens-examples/pom.xml          | 2 +-
 lens-ml-dist/pom.xml           | 2 +-
 lens-ml-lib/pom.xml            | 2 +-
 lens-query-lib/pom.xml         | 2 +-
 lens-regression/pom.xml        | 2 +-
 lens-server-api/pom.xml        | 2 +-
 lens-server/pom.xml            | 2 +-
 lens-ship-jars/pom.xml         | 2 +-
 lens-storage-db/pom.xml        | 2 +-
 lens-ui/pom.xml                | 2 +-
 pom.xml                        | 8 ++++----
 23 files changed, 26 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/checkstyle/pom.xml
----------------------------------------------------------------------
diff --git a/checkstyle/pom.xml b/checkstyle/pom.xml
index a46b4a1..464c935 100644
--- a/checkstyle/pom.xml
+++ b/checkstyle/pom.xml
@@ -29,7 +29,7 @@
   <groupId>org.apache.lens</groupId>
   <artifactId>checkstyle</artifactId>
   <name>Lens Checkstyle Rules</name>
-  <version>2.7.1</version>
+  <version>2.7.2-SNAPSHOT</version>
 
   <build>
     <plugins>
@@ -49,8 +49,4 @@
       </plugin>
     </plugins>
   </build>
-
-  <scm>
-    <tag>apache-lens-2.7.1</tag>
-  </scm>
 </project>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/contrib/clients/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/clients/pom.xml b/contrib/clients/pom.xml
index c648626..2bad7eb 100644
--- a/contrib/clients/pom.xml
+++ b/contrib/clients/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>lens-contrib</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-contrib-clients</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/contrib/clients/python/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/clients/python/pom.xml b/contrib/clients/python/pom.xml
index c233ce5..2084d96 100644
--- a/contrib/clients/python/pom.xml
+++ b/contrib/clients/python/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>lens-contrib-clients</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-python-client</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/contrib/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/pom.xml b/contrib/pom.xml
index cd44fa6..5e9c4dc 100644
--- a/contrib/pom.xml
+++ b/contrib/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-contrib</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-api/pom.xml
----------------------------------------------------------------------
diff --git a/lens-api/pom.xml b/lens-api/pom.xml
index 58ab1be..ee96fa7 100644
--- a/lens-api/pom.xml
+++ b/lens-api/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-api</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-cli/pom.xml
----------------------------------------------------------------------
diff --git a/lens-cli/pom.xml b/lens-cli/pom.xml
index f9e758c..a343d1b 100644
--- a/lens-cli/pom.xml
+++ b/lens-cli/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-client/pom.xml
----------------------------------------------------------------------
diff --git a/lens-client/pom.xml b/lens-client/pom.xml
index 9af3903..d8b4b4f 100644
--- a/lens-client/pom.xml
+++ b/lens-client/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-cube/pom.xml
----------------------------------------------------------------------
diff --git a/lens-cube/pom.xml b/lens-cube/pom.xml
index 074a593..706a288 100644
--- a/lens-cube/pom.xml
+++ b/lens-cube/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-cube</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-dist/pom.xml
----------------------------------------------------------------------
diff --git a/lens-dist/pom.xml b/lens-dist/pom.xml
index 10eb7c3..2afa04d 100644
--- a/lens-dist/pom.xml
+++ b/lens-dist/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-dist</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-driver-es/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-es/pom.xml b/lens-driver-es/pom.xml
index e20505b..e5c0a62 100644
--- a/lens-driver-es/pom.xml
+++ b/lens-driver-es/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache.lens</groupId>
     <artifactId>apache-lens</artifactId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-driver-es</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-driver-hive/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-hive/pom.xml b/lens-driver-hive/pom.xml
index a3ad080..e4d3ce4 100644
--- a/lens-driver-hive/pom.xml
+++ b/lens-driver-hive/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-driver-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-jdbc/pom.xml b/lens-driver-jdbc/pom.xml
index 22d1095..cdd6e38 100644
--- a/lens-driver-jdbc/pom.xml
+++ b/lens-driver-jdbc/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-driver-jdbc</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-examples/pom.xml
----------------------------------------------------------------------
diff --git a/lens-examples/pom.xml b/lens-examples/pom.xml
index 1af8fcf..1866d04 100644
--- a/lens-examples/pom.xml
+++ b/lens-examples/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-ml-dist/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ml-dist/pom.xml b/lens-ml-dist/pom.xml
index 15bc2a2..a431a41 100644
--- a/lens-ml-dist/pom.xml
+++ b/lens-ml-dist/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
     
   <artifactId>lens-ml-dist</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-ml-lib/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ml-lib/pom.xml b/lens-ml-lib/pom.xml
index 58f9844..143373e 100644
--- a/lens-ml-lib/pom.xml
+++ b/lens-ml-lib/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-query-lib/pom.xml
----------------------------------------------------------------------
diff --git a/lens-query-lib/pom.xml b/lens-query-lib/pom.xml
index 838dac4..3a8b33c 100644
--- a/lens-query-lib/pom.xml
+++ b/lens-query-lib/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-query-lib</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-regression/pom.xml
----------------------------------------------------------------------
diff --git a/lens-regression/pom.xml b/lens-regression/pom.xml
index 8aeed6b..89cf44e 100644
--- a/lens-regression/pom.xml
+++ b/lens-regression/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-regression</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-server-api/pom.xml
----------------------------------------------------------------------
diff --git a/lens-server-api/pom.xml b/lens-server-api/pom.xml
index 347ae39..d04e1f9 100644
--- a/lens-server-api/pom.xml
+++ b/lens-server-api/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-server-api</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-server/pom.xml
----------------------------------------------------------------------
diff --git a/lens-server/pom.xml b/lens-server/pom.xml
index 1badfb0..cb4ed94 100644
--- a/lens-server/pom.xml
+++ b/lens-server/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-ship-jars/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ship-jars/pom.xml b/lens-ship-jars/pom.xml
index 9725434..f6bc69e 100644
--- a/lens-ship-jars/pom.xml
+++ b/lens-ship-jars/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-ship-jars</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-storage-db/pom.xml
----------------------------------------------------------------------
diff --git a/lens-storage-db/pom.xml b/lens-storage-db/pom.xml
index 1cb6e09..c9629a0 100644
--- a/lens-storage-db/pom.xml
+++ b/lens-storage-db/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-storage-db</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/lens-ui/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ui/pom.xml b/lens-ui/pom.xml
index 12a2136..4b4552d 100644
--- a/lens-ui/pom.xml
+++ b/lens-ui/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.1</version>
+    <version>2.7.2-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-ui</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/865d2026/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b43a2fa..7010ea8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -128,7 +128,7 @@
 
   <groupId>org.apache.lens</groupId>
   <artifactId>apache-lens</artifactId>
-  <version>2.7.1</version>
+  <version>2.7.2-SNAPSHOT</version>
   <name>Lens</name>
   <packaging>pom</packaging>
   <description>Unified Analytics Platform</description>
@@ -379,7 +379,7 @@
     <connection>scm:git:https://git-wip-us.apache.org/repos/asf/lens.git</connection>
     <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/lens.git</developerConnection>
     <url>https://git-wip-us.apache.org/repos/asf/lens.git</url>
-    <tag>apache-lens-2.7.1</tag>
+    <tag>HEAD</tag>
   </scm>
 
   <reporting>
@@ -740,7 +740,7 @@
           <dependency>
             <groupId>org.apache.lens</groupId>
             <artifactId>checkstyle</artifactId>
-            <version>2.7.1</version>
+            <version>2.7.2-SNAPSHOT</version>
           </dependency>
         </dependencies>
         <executions>
@@ -1854,7 +1854,7 @@
               <dependency>
                 <groupId>org.apache.lens</groupId>
                 <artifactId>checkstyle</artifactId>
-                <version>2.7.1</version>
+                <version>2.7.2-SNAPSHOT</version>
               </dependency>
             </dependencies>
             <executions>


[27/50] lens git commit: LENS-1459: Time union candidate creation phase is creating a combination with redundant candidates

Posted by ra...@apache.org.
LENS-1459: Time union candidate creation phase is creating a combination with redundant candidates


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

Branch: refs/heads/master
Commit: 10eef27a0dcbb1a3b918ec0ceeabefd206e13045
Parents: 6a45685
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Wed Aug 16 17:47:39 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Mon Aug 21 13:01:04 2017 +0530

----------------------------------------------------------------------
 .../org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/10eef27a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
index de30703..1cff4a4 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/CandidateCoveringSetsResolver.java
@@ -127,6 +127,7 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
     // Get all covering fact sets
 //    List<UnionCandidate> unionCoveringSet = getCombinations(new ArrayList<>(allCandidatesPartiallyValid), cubeql);
     List<UnionCandidate> unionCoveringSet = getCombinationTailIterative(allCandidatesPartiallyValid, cubeql);
+    pruneRedundantUnionCoveringSets(unionCoveringSet);
     // Sort the Collection based on no of elements
     unionCoveringSet.sort(Comparator.comparing(Candidate::getChildrenCount));
     // prune candidate set which doesn't contain any common measure i
@@ -154,7 +155,7 @@ public class CandidateCoveringSetsResolver implements ContextRewriter {
       }
     }
   }
-  @Deprecated
+
   private void pruneRedundantUnionCoveringSets(List<UnionCandidate> candidates) {
     for (int i = 0; i < candidates.size(); i++) {
       UnionCandidate current = candidates.get(i);


[44/50] lens git commit: LENS-1472: Populate sample metastore on example db is failing

Posted by ra...@apache.org.
LENS-1472: Populate sample metastore on example db is failing


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

Branch: refs/heads/master
Commit: dcbe18e6bd967cfc5abe14c8fb59bd0bddc2a753
Parents: 6d49513
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Tue Sep 5 22:49:00 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Mon Jan 8 09:44:53 2018 +0800

----------------------------------------------------------------------
 lens-dist/src/main/assembly/bin-dist.xml                     | 8 ++++++++
 .../org/apache/lens/examples/PopulateSampleMetastore.java    | 5 +++++
 .../main/java/org/apache/lens/examples/SampleMetastore.java  | 3 +++
 .../src/main/resources/sales-aggr-fact2-local4.data          | 2 ++
 4 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/dcbe18e6/lens-dist/src/main/assembly/bin-dist.xml
----------------------------------------------------------------------
diff --git a/lens-dist/src/main/assembly/bin-dist.xml b/lens-dist/src/main/assembly/bin-dist.xml
index c3f8f17..36783b1 100644
--- a/lens-dist/src/main/assembly/bin-dist.xml
+++ b/lens-dist/src/main/assembly/bin-dist.xml
@@ -331,6 +331,14 @@
 
     <fileSet>
       <directory>../lens-examples/src/main/resources/</directory>
+      <outputDirectory>/client/examples/data/sales_aggr_fact2_local4</outputDirectory>
+      <includes>
+        <include>sales-aggr-fact2-local4*.data</include>
+      </includes>
+    </fileSet>
+
+    <fileSet>
+      <directory>../lens-examples/src/main/resources/</directory>
       <outputDirectory>/client/examples/data/sales_raw_local1</outputDirectory>
       <includes>
         <include>sales-raw-local1*.data</include>

http://git-wip-us.apache.org/repos/asf/lens/blob/dcbe18e6/lens-examples/src/main/java/org/apache/lens/examples/PopulateSampleMetastore.java
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/java/org/apache/lens/examples/PopulateSampleMetastore.java b/lens-examples/src/main/java/org/apache/lens/examples/PopulateSampleMetastore.java
index 4fc15a6..7c8fd2c 100644
--- a/lens-examples/src/main/java/org/apache/lens/examples/PopulateSampleMetastore.java
+++ b/lens-examples/src/main/java/org/apache/lens/examples/PopulateSampleMetastore.java
@@ -43,6 +43,10 @@ public class PopulateSampleMetastore {
   private static final SimpleDateFormat FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
   private static final String NOW_TIME = FORMAT.format(DATE);
 
+  private static final String CREATE_QUERY = "CREATE TABLE if not exists mydb_sales_aggr_continuous_fact "
+    + "(order_time timestamp, delivery_time timestamp, customer_id integer, product_id integer, promotion_id integer, "
+    + "customer_city_id integer, production_city_id integer, delivery_city_id integer, unit_sales integer, "
+    + "store_sales integer, store_cost integer, max_line_item_price integer, max_line_item_discount integer)";
   private static final String INSERT_QUERY = "INSERT INTO "
       + "mydb_sales_aggr_continuous_fact (order_time, delivery_time, customer_id, "
       + "product_id, promotion_id, customer_city_id, production_city_id, delivery_city_id, unit_sales, "
@@ -119,6 +123,7 @@ public class PopulateSampleMetastore {
     con.setAutoCommit(true);
     Statement statement = con.createStatement();
     try {
+      statement.execute(CREATE_QUERY);
       statement.execute(INSERT_QUERY);
 
     } finally {

http://git-wip-us.apache.org/repos/asf/lens/blob/dcbe18e6/lens-examples/src/main/java/org/apache/lens/examples/SampleMetastore.java
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/java/org/apache/lens/examples/SampleMetastore.java b/lens-examples/src/main/java/org/apache/lens/examples/SampleMetastore.java
index e5a397b..1aa993b 100644
--- a/lens-examples/src/main/java/org/apache/lens/examples/SampleMetastore.java
+++ b/lens-examples/src/main/java/org/apache/lens/examples/SampleMetastore.java
@@ -79,6 +79,9 @@ public class SampleMetastore {
   public void createCubes() throws JAXBException, IOException {
     createCube("sample-cube.xml");
     createCube("sales-cube.xml");
+    createCube("cube11.xml");
+    createCube("cube22.xml");
+    createCube("cube33.xml");
   }
 
   private void createDimension(String dimensionSpec) {

http://git-wip-us.apache.org/repos/asf/lens/blob/dcbe18e6/lens-examples/src/main/resources/sales-aggr-fact2-local4.data
----------------------------------------------------------------------
diff --git a/lens-examples/src/main/resources/sales-aggr-fact2-local4.data b/lens-examples/src/main/resources/sales-aggr-fact2-local4.data
new file mode 100644
index 0000000..407b097
--- /dev/null
+++ b/lens-examples/src/main/resources/sales-aggr-fact2-local4.data
@@ -0,0 +1,2 @@
+2015-04-13 03:00:00,2015-04-13 04:00:00,1,1,1,1,1,1,5,0
+2015-04-13 03:00:00,2015-04-13 04:00:00,2,1,2,2,2,1,8,2
\ No newline at end of file


[32/50] lens git commit: LENS-1475 : Override equals(Object obj) and hashcode() methods of SegmentationCandidate

Posted by ra...@apache.org.
LENS-1475 : Override equals(Object obj) and hashcode() methods of SegmentationCandidate


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

Branch: refs/heads/master
Commit: 0f58445e0e2bf4262a624f006c678786a00485d5
Parents: ae6bda3
Author: Sushil Mohanty <su...@gmail.com>
Authored: Thu Sep 14 12:42:57 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Thu Oct 5 11:12:29 2017 +0800

----------------------------------------------------------------------
 .../lens/cube/parse/SegmentationCandidate.java  | 21 ++++++++++++++++++++
 1 file changed, 21 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/0f58445e/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
index 055d6ef..9378009 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
@@ -404,4 +404,25 @@ public class SegmentationCandidate implements Candidate {
     return cubeQueryContextMap.entrySet().stream().filter(entry -> entry.getValue().getPickedCandidate() == null)
       .collect(toMap(Map.Entry::getKey, entry -> entry.getValue().getStoragePruningMsgs()));
   }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (super.equals(obj)) {
+      return true;
+    }
+
+    if (obj == null || !(obj instanceof SegmentationCandidate)) {
+      return false;
+    }
+
+    SegmentationCandidate segmantationCandidate = (SegmentationCandidate) obj;
+    return (segmantationCandidate.segmentation.getSegments().equals(this.segmentation.getSegments())
+        && segmantationCandidate.segmentation.getBaseCube().equals(this.segmentation.getBaseCube()));
+  }
+
+  @Override
+  public int hashCode() {
+    return segmentation.hashCode();
+  }
+
 }


[09/50] lens git commit: LENS-1443: Fallback ranges not working for virtual facts

Posted by ra...@apache.org.
LENS-1443: Fallback ranges not working for virtual facts


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

Branch: refs/heads/master
Commit: c174583ff7946d0347e0a0a87272f42c0023aecf
Parents: 9da5b40
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Wed Jun 21 13:20:09 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:42:50 2017 +0530

----------------------------------------------------------------------
 .../apache/lens/cube/metadata/FactTable.java    | 38 ++++++++++----------
 .../apache/lens/cube/parse/JoinCandidate.java   |  7 ++--
 .../lens/cube/parse/LeastPartitionResolver.java | 21 +++++------
 .../lens/cube/parse/StorageCandidate.java       |  2 +-
 .../lens/cube/parse/StorageTableResolver.java   |  5 +--
 5 files changed, 37 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/c174583f/lens-cube/src/main/java/org/apache/lens/cube/metadata/FactTable.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/FactTable.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/FactTable.java
index f87cf44..a463c47 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/FactTable.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/FactTable.java
@@ -35,132 +35,132 @@ public interface FactTable extends Named {
    *
    * @return Map of storage to set of update periods
    */
-  public Map<String, Set<UpdatePeriod>> getUpdatePeriods();
+  Map<String, Set<UpdatePeriod>> getUpdatePeriods();
 
   /**
    * Cube to which this fact belongs to
    *
    * @return the cube string
    */
-  public String getCubeName();
+  String getCubeName();
 
   /**
    * The set of Storage names
    *
    * @return set of strings
    */
-  public Set<String> getStorages();
+  Set<String> getStorages();
 
   /**
    *The type of the fact
    *
    * @return table type {@link CubeTableType}
    */
-  public CubeTableType getTableType();
+  CubeTableType getTableType();
 
   /**
    * Config properties
    *
    * @return map of string, string
    */
-  public Map<String, String> getProperties();
+  Map<String, String> getProperties();
 
   /**
    * Valid columns of the fact
    *
    * @return list of column names
    */
-  public Set<String> getValidColumns();
+  Set<String> getValidColumns();
 
   /**
    * Weight of the fact
    *
    * @return weight of the fact in double
    */
-  public double weight();
+  double weight();
 
   /**
    * Set of all the columns names of the fact
    *
    * @return set of column names
    */
-  public Set<String> getAllFieldNames();
+  Set<String> getAllFieldNames();
 
   /**
    *tag for checking data completeness
    *
    * @return Tag String
    */
-  public String getDataCompletenessTag();
+  String getDataCompletenessTag();
 
   /**
    * List of columns of the fact
    *
    * @return set of {@link FieldSchema}
    */
-  public List<FieldSchema> getColumns();
+  List<FieldSchema> getColumns();
 
   /**
    * Is Aggregated Fact
    *
    * @return true if fact is Aggregated , false otherwise
    */
-  public boolean isAggregated();
+  boolean isAggregated();
 
   /**
    * Absolute start time of the fact
    *
    * @return Absolute Start time of the fact {@link Date}
    */
-  public Date getAbsoluteStartTime();
+  Date getAbsoluteStartTime();
 
   /**
    * Relative start time of the fact
    *
    * @return Relative Start time of the fact {@link Date}
    */
-  public Date getRelativeStartTime();
+  Date getRelativeStartTime();
 
   /**
    * Start time of the fact
    *
    * @return Start time of the fact {@link Date}
    */
-  public Date getStartTime();
+  Date getStartTime();
 
   /**
    * Absolute end time of the fact
    *
    * @return Absolute End time of the fact {@link Date}
    */
-  public Date getAbsoluteEndTime();
+  Date getAbsoluteEndTime();
 
   /**
    * Relative End time of the Fact
    *
    * @return Relative end time of the fact {@link Date}
    */
-  public Date getRelativeEndTime();
+  Date getRelativeEndTime();
 
   /**
    * End time of the fact
    *
    * @return End time of the fact {@link Date}
    */
-  public Date getEndTime();
+  Date getEndTime();
 
   /**
    * Is Virtual Fact
    *
    * @return true if fact is a virtual fact, false otherwise
    */
-  public boolean isVirtualFact();
+  boolean isVirtualFact();
 
   /**
    * Storage name of the fact
    *
    * @return Storage name of the fact
    */
-  public String getSourceFactName();
+  String getSourceFactName();
 
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/c174583f/lens-cube/src/main/java/org/apache/lens/cube/parse/JoinCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/JoinCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/JoinCandidate.java
index d9915f4..1c2b9f8 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/JoinCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/JoinCandidate.java
@@ -91,12 +91,11 @@ public class JoinCandidate implements Candidate {
   @Override
   public boolean evaluateCompleteness(TimeRange timeRange, TimeRange parentTimeRange, boolean failOnPartialData)
     throws LensException {
+    boolean complete = true;
     for (Candidate child : children) {
-      if (!child.evaluateCompleteness(timeRange, parentTimeRange, failOnPartialData)) {
-        return false;
-      }
+      complete &= child.evaluateCompleteness(timeRange, parentTimeRange, failOnPartialData);
     }
-    return true;
+    return complete;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lens/blob/c174583f/lens-cube/src/main/java/org/apache/lens/cube/parse/LeastPartitionResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/LeastPartitionResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/LeastPartitionResolver.java
index 22b1d03..a76e9b1 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/LeastPartitionResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/LeastPartitionResolver.java
@@ -48,17 +48,18 @@ class LeastPartitionResolver implements ContextRewriter {
           factPartCount.put(candidate, parts);
         }
       }
+      if (!factPartCount.isEmpty()) {
+        double minPartitions = Collections.min(factPartCount.values());
 
-      double minPartitions = Collections.min(factPartCount.values());
-
-      for (Iterator<Candidate> i = cubeql.getCandidates().iterator(); i.hasNext();) {
-        Candidate candidate = i.next();
-        if (factPartCount.containsKey(candidate) && factPartCount.get(candidate) > minPartitions) {
-          log.info("Not considering Candidate:{} as it requires more partitions to be" + " queried:{} minimum:{}",
-            candidate, factPartCount.get(candidate), minPartitions);
-          i.remove();
-          cubeql.addCandidatePruningMsg(candidate,
-            new CandidateTablePruneCause(CandidateTablePruneCause.CandidateTablePruneCode.MORE_PARTITIONS));
+        for (Iterator<Candidate> i = cubeql.getCandidates().iterator(); i.hasNext();) {
+          Candidate candidate = i.next();
+          if (factPartCount.containsKey(candidate) && factPartCount.get(candidate) > minPartitions) {
+            log.info("Not considering Candidate:{} as it requires more partitions to be" + " queried:{} minimum:{}",
+              candidate, factPartCount.get(candidate), minPartitions);
+            i.remove();
+            cubeql.addCandidatePruningMsg(candidate,
+              new CandidateTablePruneCause(CandidateTablePruneCause.CandidateTablePruneCode.MORE_PARTITIONS));
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/lens/blob/c174583f/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
index 6e5aa4c..c6ef6d2 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidate.java
@@ -604,7 +604,7 @@ public class StorageCandidate implements Candidate, CandidateTable {
           break;
         }
       }
-      TimeRange fallBackRange = getFallbackRange(prevRange, this.getFact().getName(), cubeQueryContext);
+      TimeRange fallBackRange = getFallbackRange(prevRange, this.getFact().getSourceFactName(), cubeQueryContext);
       log.info("No partitions for range:{}. fallback range: {}", timeRange, fallBackRange);
       if (fallBackRange == null) {
         break;

http://git-wip-us.apache.org/repos/asf/lens/blob/c174583f/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageTableResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageTableResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageTableResolver.java
index d7da8cb..7f0f2d4 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageTableResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageTableResolver.java
@@ -20,6 +20,7 @@ package org.apache.lens.cube.parse;
 
 import static org.apache.lens.cube.parse.CandidateTablePruneCause.incompletePartitions;
 import static org.apache.lens.cube.parse.CandidateTablePruneCause.partitionColumnsMissing;
+import static org.apache.lens.cube.parse.StorageUtil.getFallbackRange;
 
 import java.util.*;
 
@@ -324,13 +325,13 @@ class StorageTableResolver implements ContextRewriter {
             } else if (!sc.getValidUpdatePeriods().contains(UpdatePeriod.CONTINUOUS)) {
               if (!client.partColExists(sc.getFact(), sc.getStorageName(), range.getPartitionColumn())) {
                 pruningCauseForThisTimeRange = partitionColumnsMissing(range.getPartitionColumn());
-                TimeRange fallBackRange = StorageUtil.getFallbackRange(range, sc.getFact().getName(), cubeql);
+                TimeRange fallBackRange = getFallbackRange(range, sc.getFact().getSourceFactName(), cubeql);
                 while (fallBackRange != null) {
                   pruningCauseForThisTimeRange = null;
                   if (!client.partColExists(sc.getFact(), sc.getStorageName(),
                     fallBackRange.getPartitionColumn())) {
                     pruningCauseForThisTimeRange = partitionColumnsMissing(fallBackRange.getPartitionColumn());
-                    fallBackRange = StorageUtil.getFallbackRange(fallBackRange, sc.getFact().getName(), cubeql);
+                    fallBackRange = getFallbackRange(fallBackRange, sc.getFact().getSourceFactName(), cubeql);
                   } else {
                     if (!sc.isPartiallyValidForTimeRange(fallBackRange)) {
                       pruningCauseForThisTimeRange =


[15/50] lens git commit: LENS-1450: filters are getting replicated in jdbc driver query

Posted by ra...@apache.org.
LENS-1450: filters are getting replicated in jdbc driver query


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

Branch: refs/heads/master
Commit: 503a46dc6c89598cc2ea557f2311e4bd26c3d9a1
Parents: dd3b1bd
Author: Rajitha R <ra...@gmail.com>
Authored: Wed Jul 5 16:49:07 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 13 14:42:58 2017 +0530

----------------------------------------------------------------------
 .../src/main/java/org/apache/lens/cube/parse/CubeQueryContext.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/503a46dc/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 b5330a7..fb49b6d 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
@@ -1169,7 +1169,7 @@ public class CubeQueryContext extends TracksQueriedColumns implements QueryAST,
         }
       }
     } else if (node.getParent() == null
-        && node.getToken().getType() != HiveParser.KW_AND) {
+        && node.getToken().getType() != HiveParser.KW_AND && node.getChildCount() == 0) {
       // if node is the only child
       allFilters.add(HQLParser.getString((ASTNode) node));
     }


[36/50] lens git commit: LENS-1468: Expressions in having clauses are not getting rewritten properly for join queries

Posted by ra...@apache.org.
LENS-1468: Expressions in having clauses are not getting rewritten properly for join queries


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

Branch: refs/heads/master
Commit: 2772efb275742eb4c03da5e7635bdbcfab63cebc
Parents: 9a678d8
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Sun Sep 3 18:27:59 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Thu Oct 5 11:13:31 2017 +0800

----------------------------------------------------------------------
 .../lens/cube/parse/ExpressionResolver.java     | 55 +++++++++++++++-----
 .../cube/parse/StorageCandidateHQLContext.java  |  4 ++
 .../lens/cube/parse/UnionQueryWriter.java       |  8 +++
 .../lens/cube/parse/TestBaseCubeQueries.java    | 32 +++++++++++-
 .../resources/schema/cubes/base/basecube.xml    |  6 +++
 .../resources/schema/cubes/derived/der2.xml     |  2 +
 6 files changed, 92 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/2772efb2/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
index 822e25e..553468f 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/ExpressionResolver.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.ql.parse.HiveParser;
 
 import org.antlr.runtime.CommonToken;
 
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import lombok.*;
 import lombok.extern.slf4j.Slf4j;
@@ -262,7 +263,7 @@ class ExpressionResolver implements ContextRewriter {
 
   @RequiredArgsConstructor
   @ToString
-  private static class PickedExpression {
+  static class PickedExpression {
     private final String srcAlias;
     private final ExprSpecContext pickedCtx;
     private transient ASTNode reWrittenAST = null;
@@ -286,11 +287,13 @@ class ExpressionResolver implements ContextRewriter {
 
   static class ExpressionResolverContext {
     @Getter
-    private Map<String, Set<ExpressionContext>> allExprsQueried = new HashMap<String, Set<ExpressionContext>>();
-    private Map<String, Set<PickedExpression>> pickedExpressions = new HashMap<String, Set<PickedExpression>>();
+    private Map<String, Set<ExpressionContext>> allExprsQueried = new HashMap<>();
+    private Map<String, Set<PickedExpression>> pickedExpressions = new HashMap<>();
+    @Getter
+    private Map<DimHQLContext, Map<String, Set<PickedExpression>>> pickedExpressionsPerCandidate = new HashMap<>();
     private Map<String, ASTNode> nonPickedExpressionsForCandidate = new HashMap<String, ASTNode>();
     private final CubeQueryContext cubeql;
-
+    private boolean replacedHavingExpressions = false;
     ExpressionResolverContext(CubeQueryContext cubeql) {
       this.cubeql = cubeql;
     }
@@ -411,7 +414,7 @@ class ExpressionResolver implements ContextRewriter {
         // Replace picked expressions in all the base trees
         replacePickedExpressions(sc);
       }
-
+      pickedExpressionsPerCandidate.put(sc, Maps.newHashMap(pickedExpressions));
       pickedExpressions.clear();
       nonPickedExpressionsForCandidate.clear();
 
@@ -430,16 +433,42 @@ class ExpressionResolver implements ContextRewriter {
       replaceAST(cubeql, queryAST.getJoinAST());
       replaceAST(cubeql, queryAST.getGroupByAST());
       // Resolve having expression for StorageCandidate
-      if (queryAST.getHavingAST() != null) {
-        replaceAST(cubeql, queryAST.getHavingAST());
-      } else if (cubeql.getHavingAST() != null) {
-        ASTNode havingCopy = MetastoreUtil.copyAST(cubeql.getHavingAST());
-        replaceAST(cubeql, havingCopy);
-        queryAST.setHavingAST(havingCopy);
-      }
       replaceAST(cubeql, queryAST.getOrderByAST());
     }
-
+    public void replaceHavingExpressions() throws LensException {
+      replaceHavingExpressions(pickedExpressionsPerCandidate);
+    }
+    public void replaceHavingExpressions(
+      Map<DimHQLContext, Map<String, Set<PickedExpression>>> pickedExpressionsPerCandidate) throws LensException {
+      if (cubeql.getHavingAST() != null && !replacedHavingExpressions) {
+        HQLParser.bft(cubeql.getHavingAST(), visited -> {
+          ASTNode node1 = visited.getNode();
+          int childcount = node1.getChildCount();
+          for (int i = 0; i < childcount; i++) {
+            ASTNode current = (ASTNode) node1.getChild(i);
+            if (current.getToken().getType() == DOT) {
+              // This is for the case where column name is prefixed by table name
+              // or table alias
+              // For example 'select fact.id, dim2.id ...'
+              // Right child is the column name, left child.ident is table name
+              ASTNode tabident = HQLParser.findNodeByPath(current, TOK_TABLE_OR_COL, Identifier);
+              ASTNode colIdent = (ASTNode) current.getChild(1);
+              String column = colIdent.getText().toLowerCase();
+
+              Optional<PickedExpression> exprOptional = pickedExpressionsPerCandidate.values().stream()
+                .filter(x -> x.containsKey(column)).map(x -> x.get(column)).flatMap(Collection::stream)
+                .filter(x -> x.srcAlias.equals(tabident.getText().toLowerCase())).findFirst();
+
+              if (exprOptional.isPresent()) {
+                PickedExpression expr = exprOptional.get();
+                node1.setChild(i, replaceAlias(expr.getRewrittenAST(), cubeql));
+              }
+            }
+          }
+        });
+        replacedHavingExpressions = true;
+      }
+    }
     private void replaceAST(final CubeQueryContext cubeql, ASTNode node) throws LensException {
       if (node == null) {
         return;

http://git-wip-us.apache.org/repos/asf/lens/blob/2772efb2/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
index 993aa4c..21cdb61 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
@@ -129,6 +129,10 @@ public class StorageCandidateHQLContext extends DimHQLContext {
   @Override
   protected void setMissingExpressions() throws LensException {
     setFrom(getFromTable());
+    if (getQueryAst().getHavingAST() != null) {
+      getStorageCandidate().getCubeQueryContext().getExprCtx().replaceHavingExpressions();
+      getQueryAst().setHavingAST(getCubeQueryContext().getHavingAST());
+    }
     String whereString = genWhereClauseWithDimPartitions(getWhere());
     StringBuilder whereStringBuilder = (whereString != null) ? new StringBuilder(whereString) :  new StringBuilder();
 

http://git-wip-us.apache.org/repos/asf/lens/blob/2772efb2/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
index 9dc7ee6..4eb086b 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/UnionQueryWriter.java
@@ -21,6 +21,7 @@ package org.apache.lens.cube.parse;
 
 import static java.util.stream.Collectors.joining;
 import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toSet;
 
 import static org.apache.lens.cube.parse.HQLParser.*;
 
@@ -30,6 +31,7 @@ import java.util.*;
 
 import org.apache.lens.cube.metadata.*;
 import org.apache.lens.cube.metadata.join.JoinPath;
+import org.apache.lens.cube.parse.ExpressionResolver.PickedExpression;
 import org.apache.lens.server.api.error.LensException;
 
 import org.apache.hadoop.hive.ql.lib.Node;
@@ -460,6 +462,12 @@ public class UnionQueryWriter extends SimpleHQLContext {
     queryAst.setSelectAST(outerSelectAst);
 
     // Iterate over the StorageCandidates and add non projected having columns in inner select ASTs
+    Map<DimHQLContext, Map<String, Set<PickedExpression>>> pickedExpressionsPerCandidate = new HashMap<>();
+    for (CubeQueryContext cubeQueryContext : storageCandidates.stream()
+      .map(StorageCandidateHQLContext::getCubeQueryContext).collect(toSet())) {
+      pickedExpressionsPerCandidate.putAll(cubeQueryContext.getExprCtx().getPickedExpressionsPerCandidate());
+    }
+    cubeql.getExprCtx().replaceHavingExpressions(pickedExpressionsPerCandidate);
     for (StorageCandidateHQLContext sc : storageCandidates) {
       aliasDecider.setCounter(selectAliasCounter);
       processHavingAST(sc.getQueryAst().getSelectAST(), aliasDecider, sc);

http://git-wip-us.apache.org/repos/asf/lens/blob/2772efb2/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 cf29dff..35cb2b5 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
@@ -835,13 +835,41 @@ public class TestBaseCubeQueries extends TestQueryRewrite {
       }
     }
   }
+  @Test
+  public void testHavingOnTwoExpressions() throws Exception {
+    CubeQueryContext ctx1 = rewriteCtx("select dim1, msr2 from basecube where " + TWO_DAYS_RANGE
+      + "having effectivemsr2 > 0 and complexmsr12 > 10", conf);
+    CubeQueryContext ctx2 = rewriteCtx("select dim1, msr2 from basecube where " + TWO_DAYS_RANGE
+      + "having effectivemsr2 > 0 and complexmsr12 > 10", conf);
+    // shuffle join candidate order in ctx2
+    for (Candidate candidate : ctx2.getCandidates()) {
+      if (candidate instanceof JoinCandidate) {
+        JoinCandidate jc = (JoinCandidate) candidate;
+        List<Candidate> children = jc.getChildren();
+        Collections.reverse(children);
+      }
+    }
+    // toHQL outputs are tested in other functions, not testing here.
+
+    // test having clauses are same in both
+    String having1 = ctx1.toHQL().substring(ctx1.toHQL().indexOf("HAVING"));
+    String having2 = ctx2.toHQL().substring(ctx2.toHQL().indexOf("HAVING"));
+    assertEquals(having1, having2, "having1: " + having1 + "\nhaving2: " + having2);
+
+    // assert order of facts is differnet in to hqls
+    int ind11 = ctx1.toHQL().indexOf("c1_testfact1_base");
+    int ind21 = ctx2.toHQL().indexOf("c1_testfact1_base");
+
+    int ind12 = ctx1.toHQL().indexOf("c1_testfact2_base");
+    int ind22 = ctx2.toHQL().indexOf("c1_testfact2_base");
+
+    assertTrue((ind11 < ind21 && ind12 > ind22) || (ind11 > ind21 && ind12 < ind22));
+  }
 
   @Test
   public void testMultiFactQueryWithHaving() throws Exception {
 
     String hqlQuery, expected1, expected2;
-    String endSubString = "mq2 on mq1.dim1 <=> mq2.dim1 AND mq1.dim11 <=> mq2.dim11";
-    String joinSubString = "mq1 full outer join ";
 
     // only One having clause, that too answerable from one fact
     hqlQuery = rewrite("select dim1, dim11, msr12 from basecube where " + TWO_DAYS_RANGE

http://git-wip-us.apache.org/repos/asf/lens/blob/2772efb2/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/cubes/base/basecube.xml b/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
index 6bb5eb9..6cc3201 100644
--- a/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
+++ b/lens-cube/src/test/resources/schema/cubes/base/basecube.xml
@@ -269,6 +269,12 @@
     <expression _type="double" name="flooredmsr12" display_string="Floored msr12" description="floored measure12">
       <expr_spec expr="floor(msr12)"/>
     </expression>
+    <expression _type="double" name="complexmsr12" display_string="Floored msr12" description="floored measure12">
+      <expr_spec expr="floor(msr12)+0"/>
+    </expression>
+    <expression _type="double" name="effectivemsr2" display_string="effective msr12" description="effective measure2">
+      <expr_spec expr="msr2 + msr21 + msr22"/>
+    </expression>
     <expression _type="String" name="cityandstate" display_string="City and State"
                 description="city and state together">
       <expr_spec expr="concat(cityname, &quot;:&quot;, statename_cube)"/>

http://git-wip-us.apache.org/repos/asf/lens/blob/2772efb2/lens-cube/src/test/resources/schema/cubes/derived/der2.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/schema/cubes/derived/der2.xml b/lens-cube/src/test/resources/schema/cubes/derived/der2.xml
index 337e7f4..213fe07 100644
--- a/lens-cube/src/test/resources/schema/cubes/derived/der2.xml
+++ b/lens-cube/src/test/resources/schema/cubes/derived/der2.xml
@@ -33,6 +33,8 @@
   <measure_names>
     <measure_name>directmsr</measure_name>
     <measure_name>msr2</measure_name>
+    <measure_name>msr21</measure_name>
+    <measure_name>msr22</measure_name>
     <measure_name>msr12</measure_name>
     <measure_name>msr14</measure_name>
     <measure_name>msr13</measure_name>


[43/50] lens git commit: [maven-release-plugin] prepare for next development iteration

Posted by ra...@apache.org.
[maven-release-plugin] prepare for next development iteration


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

Branch: refs/heads/master
Commit: 6d49513e0f75f334fb093d59bf2aa72533930558
Parents: d2b4992
Author: rajub <ra...@lazada.com>
Authored: Tue Dec 5 05:45:24 2017 +0800
Committer: rajub <ra...@lazada.com>
Committed: Tue Dec 5 05:45:24 2017 +0800

----------------------------------------------------------------------
 checkstyle/pom.xml             | 6 +-----
 contrib/clients/pom.xml        | 2 +-
 contrib/clients/python/pom.xml | 2 +-
 contrib/pom.xml                | 2 +-
 lens-api/pom.xml               | 2 +-
 lens-cli/pom.xml               | 2 +-
 lens-client/pom.xml            | 2 +-
 lens-cube/pom.xml              | 2 +-
 lens-dist/pom.xml              | 2 +-
 lens-driver-es/pom.xml         | 2 +-
 lens-driver-hive/pom.xml       | 2 +-
 lens-driver-jdbc/pom.xml       | 2 +-
 lens-examples/pom.xml          | 2 +-
 lens-ml-dist/pom.xml           | 2 +-
 lens-ml-lib/pom.xml            | 2 +-
 lens-query-lib/pom.xml         | 2 +-
 lens-regression/pom.xml        | 2 +-
 lens-server-api/pom.xml        | 2 +-
 lens-server/pom.xml            | 2 +-
 lens-ship-jars/pom.xml         | 2 +-
 lens-storage-db/pom.xml        | 2 +-
 lens-ui/pom.xml                | 2 +-
 pom.xml                        | 8 ++++----
 23 files changed, 26 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/checkstyle/pom.xml
----------------------------------------------------------------------
diff --git a/checkstyle/pom.xml b/checkstyle/pom.xml
index db378e2..0076790 100644
--- a/checkstyle/pom.xml
+++ b/checkstyle/pom.xml
@@ -29,7 +29,7 @@
   <groupId>org.apache.lens</groupId>
   <artifactId>checkstyle</artifactId>
   <name>Lens Checkstyle Rules</name>
-  <version>2.7.0</version>
+  <version>2.7.1-SNAPSHOT</version>
 
   <build>
     <plugins>
@@ -49,8 +49,4 @@
       </plugin>
     </plugins>
   </build>
-
-  <scm>
-    <tag>apache-lens-2.7.0</tag>
-  </scm>
 </project>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/contrib/clients/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/clients/pom.xml b/contrib/clients/pom.xml
index 4eeba9a..260cac7 100644
--- a/contrib/clients/pom.xml
+++ b/contrib/clients/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>lens-contrib</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-contrib-clients</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/contrib/clients/python/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/clients/python/pom.xml b/contrib/clients/python/pom.xml
index f3969cb..039d29d 100644
--- a/contrib/clients/python/pom.xml
+++ b/contrib/clients/python/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>lens-contrib-clients</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-python-client</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/contrib/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/pom.xml b/contrib/pom.xml
index 7ce7982..d3d14e8 100644
--- a/contrib/pom.xml
+++ b/contrib/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-contrib</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-api/pom.xml
----------------------------------------------------------------------
diff --git a/lens-api/pom.xml b/lens-api/pom.xml
index ab3cd25..b36e3c5 100644
--- a/lens-api/pom.xml
+++ b/lens-api/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-api</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-cli/pom.xml
----------------------------------------------------------------------
diff --git a/lens-cli/pom.xml b/lens-cli/pom.xml
index e0e0b53..b09d826 100644
--- a/lens-cli/pom.xml
+++ b/lens-cli/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-client/pom.xml
----------------------------------------------------------------------
diff --git a/lens-client/pom.xml b/lens-client/pom.xml
index bc85772..1591fb2 100644
--- a/lens-client/pom.xml
+++ b/lens-client/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-cube/pom.xml
----------------------------------------------------------------------
diff --git a/lens-cube/pom.xml b/lens-cube/pom.xml
index 855daba..7a7cb03 100644
--- a/lens-cube/pom.xml
+++ b/lens-cube/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-cube</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-dist/pom.xml
----------------------------------------------------------------------
diff --git a/lens-dist/pom.xml b/lens-dist/pom.xml
index e82a5a1..22b6ca5 100644
--- a/lens-dist/pom.xml
+++ b/lens-dist/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-dist</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-driver-es/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-es/pom.xml b/lens-driver-es/pom.xml
index f64846a..5b3f61f 100644
--- a/lens-driver-es/pom.xml
+++ b/lens-driver-es/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache.lens</groupId>
     <artifactId>apache-lens</artifactId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-driver-es</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-driver-hive/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-hive/pom.xml b/lens-driver-hive/pom.xml
index ef95d7a..08ecff2 100644
--- a/lens-driver-hive/pom.xml
+++ b/lens-driver-hive/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-driver-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/lens-driver-jdbc/pom.xml b/lens-driver-jdbc/pom.xml
index c68d0c6..d2164f2 100644
--- a/lens-driver-jdbc/pom.xml
+++ b/lens-driver-jdbc/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-driver-jdbc</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-examples/pom.xml
----------------------------------------------------------------------
diff --git a/lens-examples/pom.xml b/lens-examples/pom.xml
index 25807a8..c9c11a8 100644
--- a/lens-examples/pom.xml
+++ b/lens-examples/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-ml-dist/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ml-dist/pom.xml b/lens-ml-dist/pom.xml
index b737028..8a744e5 100644
--- a/lens-ml-dist/pom.xml
+++ b/lens-ml-dist/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
     
   <artifactId>lens-ml-dist</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-ml-lib/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ml-lib/pom.xml b/lens-ml-lib/pom.xml
index e2a809e..cc587bb 100644
--- a/lens-ml-lib/pom.xml
+++ b/lens-ml-lib/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-query-lib/pom.xml
----------------------------------------------------------------------
diff --git a/lens-query-lib/pom.xml b/lens-query-lib/pom.xml
index 017cb98..558553d 100644
--- a/lens-query-lib/pom.xml
+++ b/lens-query-lib/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-query-lib</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-regression/pom.xml
----------------------------------------------------------------------
diff --git a/lens-regression/pom.xml b/lens-regression/pom.xml
index fbcb35b..7cde8f0 100644
--- a/lens-regression/pom.xml
+++ b/lens-regression/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-regression</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-server-api/pom.xml
----------------------------------------------------------------------
diff --git a/lens-server-api/pom.xml b/lens-server-api/pom.xml
index f3750eb..5ac73da 100644
--- a/lens-server-api/pom.xml
+++ b/lens-server-api/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-server-api</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-server/pom.xml
----------------------------------------------------------------------
diff --git a/lens-server/pom.xml b/lens-server/pom.xml
index 807e70e..0d99138 100644
--- a/lens-server/pom.xml
+++ b/lens-server/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-ship-jars/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ship-jars/pom.xml b/lens-ship-jars/pom.xml
index a3c37cf..1d3cd6e 100644
--- a/lens-ship-jars/pom.xml
+++ b/lens-ship-jars/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-ship-jars</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-storage-db/pom.xml
----------------------------------------------------------------------
diff --git a/lens-storage-db/pom.xml b/lens-storage-db/pom.xml
index 1b3f9bd..544ef29 100644
--- a/lens-storage-db/pom.xml
+++ b/lens-storage-db/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-storage-db</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/lens-ui/pom.xml
----------------------------------------------------------------------
diff --git a/lens-ui/pom.xml b/lens-ui/pom.xml
index 88a2974..55320c6 100644
--- a/lens-ui/pom.xml
+++ b/lens-ui/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>apache-lens</artifactId>
     <groupId>org.apache.lens</groupId>
-    <version>2.7.0</version>
+    <version>2.7.1-SNAPSHOT</version>
   </parent>
 
   <artifactId>lens-ui</artifactId>

http://git-wip-us.apache.org/repos/asf/lens/blob/6d49513e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 244beaa..d20f105 100644
--- a/pom.xml
+++ b/pom.xml
@@ -127,7 +127,7 @@
 
   <groupId>org.apache.lens</groupId>
   <artifactId>apache-lens</artifactId>
-  <version>2.7.0</version>
+  <version>2.7.1-SNAPSHOT</version>
   <name>Lens</name>
   <packaging>pom</packaging>
   <description>Unified Analytics Platform</description>
@@ -378,7 +378,7 @@
     <connection>scm:git:https://git-wip-us.apache.org/repos/asf/lens.git</connection>
     <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/lens.git</developerConnection>
     <url>https://git-wip-us.apache.org/repos/asf/lens.git</url>
-    <tag>apache-lens-2.7.0</tag>
+    <tag>HEAD</tag>
   </scm>
 
   <reporting>
@@ -732,7 +732,7 @@
           <dependency>
             <groupId>org.apache.lens</groupId>
             <artifactId>checkstyle</artifactId>
-            <version>2.7.0</version>
+            <version>2.7.1-SNAPSHOT</version>
           </dependency>
         </dependencies>
         <executions>
@@ -1846,7 +1846,7 @@
               <dependency>
                 <groupId>org.apache.lens</groupId>
                 <artifactId>checkstyle</artifactId>
-                <version>2.7.0</version>
+                <version>2.7.1-SNAPSHOT</version>
               </dependency>
             </dependencies>
             <executions>


[35/50] lens git commit: LENS-1467: CubeQueryContext.getAllFilters is returning incorrect list of filters in case there is an "OR" in the filters

Posted by ra...@apache.org.
LENS-1467: CubeQueryContext.getAllFilters is returning incorrect list of filters in case there is an "OR" in the filters


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

Branch: refs/heads/master
Commit: 9a678d8c2602860aac81bb31801c6c6acb946054
Parents: 13cbc81
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Wed Aug 30 16:10:54 2017 +0530
Committer: rajub <ra...@lazada.com>
Committed: Thu Oct 5 11:13:18 2017 +0800

----------------------------------------------------------------------
 .../lens/cube/parse/CubeQueryContext.java       | 67 ++++++-------------
 .../cube/parse/StorageCandidateHQLContext.java  |  3 +-
 .../apache/lens/cube/parse/join/JoinClause.java |  9 +++
 .../lens/cube/parse/CubeQueryContextTest.java   | 70 ++++++++++++++++++++
 4 files changed, 100 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/9a678d8c/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 8b9583a..bff5c47 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
@@ -49,7 +49,6 @@ import org.apache.lens.cube.metadata.*;
 import org.apache.lens.cube.metadata.join.TableRelationship;
 import org.apache.lens.cube.parse.join.AutoJoinContext;
 import org.apache.lens.cube.parse.join.JoinClause;
-import org.apache.lens.cube.parse.join.JoinTree;
 import org.apache.lens.cube.parse.join.JoinUtils;
 import org.apache.lens.server.api.error.LensException;
 
@@ -1137,14 +1136,13 @@ public class CubeQueryContext extends TracksQueriedColumns implements QueryAST,
     return ImmutableSet.copyOf(this.queriedTimeDimCols);
   }
 
-  String getWhere(StorageCandidateHQLContext sc, AutoJoinContext autoJoinCtx,
-    ASTNode node, String cubeAlias,
-    boolean shouldReplaceDimFilter, String storageTable,
-    Map<Dimension, CandidateDim> dimToQuery) throws LensException {
+  String getWhere(StorageCandidateHQLContext sc, AutoJoinContext autoJoinCtx, String cubeAlias,
+    boolean shouldReplaceDimFilter, Map<Dimension, CandidateDim> dimToQuery) throws LensException {
     String whereString;
     if (autoJoinCtx != null && shouldReplaceDimFilter) {
       List<String> allfilters = new ArrayList<>();
-      getAllFilters(node, cubeAlias, allfilters, autoJoinCtx.getJoinClause(sc.getStorageCandidate()), dimToQuery);
+      getAllFilters(sc.getQueryAst().getWhereAST(), cubeAlias, allfilters,
+        autoJoinCtx.getJoinClause(sc.getStorageCandidate()), dimToQuery);
       whereString = StringUtils.join(allfilters, " and ");
     } else {
       whereString = HQLParser.getString(sc.getQueryAst().getWhereAST());
@@ -1152,58 +1150,33 @@ public class CubeQueryContext extends TracksQueriedColumns implements QueryAST,
     return whereString;
   }
 
-  private void getAllFilters(ASTNode node, String cubeAlias, List<String> allFilters,
-                                    JoinClause joinClause,  Map<Dimension, CandidateDim> dimToQuery)
-    throws LensException {
-
-    if (node.getToken().getType() == HiveParser.KW_AND) {
-      // left child is and
-      if (node.getChild(0).getType() == HiveParser.KW_AND) {
-        // take right corresponding to right
-        String table = getTableFromFilterAST((ASTNode) node.getChild(1));
-        allFilters.add(getFilter(table, cubeAlias, node, joinClause, 1, dimToQuery));
-      } else if (node.getChildCount() > 1) {
-        for (int i = 0; i < node.getChildCount(); i++) {
-          String table = getTableFromFilterAST((ASTNode) node.getChild(i));
-          allFilters.add(getFilter(table, cubeAlias, node, joinClause, i, dimToQuery));
-        }
+  protected static void getAllFilters(ASTNode node, String cubeAlias, List<String> allFilters, JoinClause joinClause,
+    Map<Dimension, CandidateDim> dimToQuery) throws LensException {
+    if (node.getToken().getType() == HiveParser.KW_AND || node.getToken().getType() == HiveParser.TOK_WHERE) {
+      for (int i = 0; i < node.getChildCount(); i++) {
+        ASTNode child = (ASTNode) node.getChild(i);
+        getAllFilters(child, cubeAlias, allFilters, joinClause, dimToQuery);
       }
-    } else if (node.getParent() == null
-        && node.getToken().getType() != HiveParser.KW_AND
-      && node.getChild(0).getType() != HiveParser.KW_AND) {
-      // if node is the only child
-      allFilters.add(HQLParser.getString((ASTNode) node));
-    }
-    for (int i = 0; i < node.getChildCount(); i++) {
-      ASTNode child = (ASTNode) node.getChild(i);
-      getAllFilters(child, cubeAlias, allFilters, joinClause, dimToQuery);
+    } else {
+      String table = getTableFromFilterAST(node);
+      allFilters.add(getFilter(table, cubeAlias, node, joinClause, dimToQuery));
     }
   }
 
-  private String getFilter(String table, String cubeAlias, ASTNode node,  JoinClause joinClause,
-                           int index,  Map<Dimension, CandidateDim> dimToQuery)
+  private static String getFilter(String table, String cubeAlias, ASTNode node,  JoinClause joinClause,
+                           Map<Dimension, CandidateDim> dimToQuery)
     throws LensException{
     String filter;
-    if (table != null && !table.equals(cubeAlias) && getStarJoin(joinClause, table) != null) {
+    if (table != null && !table.equals(cubeAlias) && joinClause.getStarJoin(table) != null) {
       //rewrite dim filter to fact filter if its a star join with fact
-      filter = buildFactSubqueryFromDimFilter(getStarJoin(joinClause, table),
-          (ASTNode) node.getChild(index), table, dimToQuery, cubeAlias);
+      filter = buildFactSubqueryFromDimFilter(joinClause.getStarJoin(table), node, table, dimToQuery, cubeAlias);
     } else {
-      filter = HQLParser.getString((ASTNode) node.getChild(index));
+      filter = HQLParser.getString(node);
     }
     return filter;
   }
 
-  private TableRelationship getStarJoin(JoinClause joinClause, String table) {
-    for (Map.Entry<TableRelationship, JoinTree>  entry : joinClause.getJoinTree().getSubtrees().entrySet()) {
-      if (entry.getValue().getDepthFromRoot() == 1 && table.equals(entry.getValue().getAlias())) {
-        return entry.getKey();
-      }
-    }
-    return null;
-  }
-
-  private String getTableFromFilterAST(ASTNode node) {
+  private static String getTableFromFilterAST(ASTNode node) {
 
     if (node.getToken().getType() == HiveParser.DOT) {
       ASTNode n = HQLParser.findNodeByPath(node, TOK_TABLE_OR_COL, Identifier);
@@ -1222,7 +1195,7 @@ public class CubeQueryContext extends TracksQueriedColumns implements QueryAST,
     return null;
   }
 
-  private String buildFactSubqueryFromDimFilter(TableRelationship tabRelation, ASTNode dimFilter,
+  private static String buildFactSubqueryFromDimFilter(TableRelationship tabRelation, ASTNode dimFilter,
                                                 String dimAlias, Map<Dimension, CandidateDim> dimToQuery,
                                                 String cubeAlias)
     throws LensException {

http://git-wip-us.apache.org/repos/asf/lens/blob/9a678d8c/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
index 494b08e..993aa4c 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/StorageCandidateHQLContext.java
@@ -90,9 +90,8 @@ public class StorageCandidateHQLContext extends DimHQLContext {
       String qualifiedStorageTable = getStorageCandidate().getStorageName();
       String storageTable = qualifiedStorageTable.substring(qualifiedStorageTable.indexOf(".") + 1);
       String where = getCubeQueryContext().getWhere(this, getCubeQueryContext().getAutoJoinCtx(),
-        getQueryAst().getWhereAST(),
         getCubeQueryContext().getAliasForTableName(getStorageCandidate().getBaseTable().getName()),
-        getCubeQueryContext().shouldReplaceDimFilterWithFactFilter(), storageTable, getDimsToQuery());
+        getCubeQueryContext().shouldReplaceDimFilterWithFactFilter(), getDimsToQuery());
       setWhere(where);
     }
   }

http://git-wip-us.apache.org/repos/asf/lens/blob/9a678d8c/lens-cube/src/main/java/org/apache/lens/cube/parse/join/JoinClause.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/join/JoinClause.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/join/JoinClause.java
index 8661496..9e8f9bc 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/join/JoinClause.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/join/JoinClause.java
@@ -50,6 +50,15 @@ public class JoinClause implements Comparable<JoinClause> {
     this.dimsInPath = dimsInPath;
   }
 
+  public TableRelationship getStarJoin(String table) {
+    for (Map.Entry<TableRelationship, JoinTree>  entry : getJoinTree().getSubtrees().entrySet()) {
+      if (entry.getValue().getDepthFromRoot() == 1 && table.equals(entry.getValue().getAlias())) {
+        return entry.getKey();
+      }
+    }
+    return null;
+  }
+
   void initChainColumns() {
     for (List<TableRelationship> path : chain.values()) {
       for (TableRelationship edge : path) {

http://git-wip-us.apache.org/repos/asf/lens/blob/9a678d8c/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeQueryContextTest.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeQueryContextTest.java b/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeQueryContextTest.java
new file mode 100644
index 0000000..19e4f44
--- /dev/null
+++ b/lens-cube/src/test/java/org/apache/lens/cube/parse/CubeQueryContextTest.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.cube.parse;
+
+import static org.testng.Assert.assertEquals;
+
+import static com.google.common.collect.Lists.newArrayList;
+
+import java.util.List;
+
+import org.apache.lens.cube.parse.join.JoinClause;
+
+import org.mockito.Mockito;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Created on 28/08/17.
+ */
+public class CubeQueryContextTest {
+  private JoinClause jc;
+
+  @BeforeClass
+  public void intClass() {
+    jc = Mockito.mock(JoinClause.class);
+    Mockito.when(jc.getStarJoin(Mockito.anyString())).thenReturn(null);
+  }
+
+  @DataProvider
+  public Object[][] testCases() {
+    return new Object[][]{
+      {"testcube.x=1 and (testcube.dt=yesterday or (testcube.dt=today and testcube.pt=yesterday))",
+        newArrayList("((testcube.x) = 1)",
+          "(((testcube.dt) = yesterday) or (((testcube.dt) = today) and ((testcube.pt) = yesterday)))"), },
+      {"testcube.x=1 and (testcube.dt=yesterday and (testcube.dt=today and testcube.pt=yesterday))",
+        newArrayList("((testcube.x) = 1)", "((testcube.dt) = yesterday)",
+          "((testcube.dt) = today)", "((testcube.pt) = yesterday)"), },
+      {"testcube.x=1 and (testcube.dt = yesterday or "
+        + "(case when true and false then 1 else 0 end))",
+        newArrayList("((testcube.x) = 1)",
+          "(((testcube.dt) = yesterday) or case  when ( true  and  false ) then 1 else 0 end)"), },
+    };
+  }
+
+  @Test(dataProvider = "testCases")
+  public void testGetAllFilters(String expr, List<String> expected) throws Exception {
+    List<String> allFilters = newArrayList();
+    CubeQueryContext.getAllFilters(HQLParser.parseExpr(expr), "testcube", allFilters, jc, Maps.newHashMap());
+    assertEquals(allFilters, expected);
+  }
+}


[21/50] lens git commit: LENS-1453: Method metrics names duplicate for segmentation inner rewrite

Posted by ra...@apache.org.
LENS-1453: Method metrics names duplicate for segmentation inner rewrite


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

Branch: refs/heads/master
Commit: 65ca04b598bd29201a6670b7a51679f9ccb9ccc6
Parents: 6dca446
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Thu Jul 20 16:09:58 2017 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Thu Jul 20 16:13:07 2017 +0530

----------------------------------------------------------------------
 .../org/apache/lens/cube/parse/SegmentationCandidate.java   | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/65ca04b5/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java b/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
index a2bd485..35638f3 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/parse/SegmentationCandidate.java
@@ -52,6 +52,7 @@ import org.apache.lens.cube.metadata.MetastoreUtil;
 import org.apache.lens.cube.metadata.Segment;
 import org.apache.lens.cube.metadata.Segmentation;
 import org.apache.lens.cube.metadata.TimeRange;
+import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.error.LensException;
 
 import org.apache.commons.lang3.tuple.Pair;
@@ -141,7 +142,13 @@ public class SegmentationCandidate implements Candidate {
         });
       addCubeNameAndAlias(ast, innerCube);
       trimHavingAndOrderby(ast, innerCube);
-      CubeQueryRewriter rewriter = new CubeQueryRewriter(conf, hconf);
+      Configuration innerConf = conf;
+      if (conf.get(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY) != null) {
+        innerConf = new Configuration(conf);
+        innerConf.set(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY,
+          conf.get(LensConfConstants.QUERY_METRIC_UNIQUE_ID_CONF_KEY) + "-" + segment.getName());
+      }
+      CubeQueryRewriter rewriter = new CubeQueryRewriter(innerConf, hconf);
       CubeQueryContext ctx = rewriter.rewrite(ast);
       cubeQueryContextMap.put(segment.getName(), ctx);
       if (!ctx.getCandidates().isEmpty()) {


[45/50] lens git commit: LENS-1484: Fix enunciate plugin issue(i.e. failing with maven 3.5.2)

Posted by ra...@apache.org.
LENS-1484: Fix enunciate plugin issue(i.e. failing with maven 3.5.2)


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

Branch: refs/heads/master
Commit: 0240f6b42bab3af859cbd26eb942c041f639f573
Parents: dcbe18e
Author: rajub <ra...@lazada.com>
Authored: Mon Jan 8 14:05:02 2018 +0800
Committer: rajub <ra...@lazada.com>
Committed: Mon Jan 8 14:05:02 2018 +0800

----------------------------------------------------------------------
 pom.xml | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/0240f6b4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d20f105..a50aed3 100644
--- a/pom.xml
+++ b/pom.xml
@@ -73,6 +73,7 @@
     <subethasmtp.version>3.1.7</subethasmtp.version>
     <liquibase.version>3.0.7</liquibase.version>
     <spring.shell.version>1.1.0.RELEASE</spring.shell.version>
+    <javax.api.version>1.3</javax.api.version>
 
     <c3p0.version>0.9.5</c3p0.version>
     <hsqldb.version>2.2.9</hsqldb.version>
@@ -90,7 +91,7 @@
     <jaxb2.basics.plugin.version>0.9.5</jaxb2.basics.plugin.version>
     <jaxb2.fluent.plugin.version>3.0</jaxb2.fluent.plugin.version>
     <exec.plugin.version>1.2.1</exec.plugin.version>
-    <enunciate.plugin.version>2.5.0</enunciate.plugin.version>
+    <enunciate.plugin.version>2.6.0</enunciate.plugin.version>
     <reports.plugin.version>2.7</reports.plugin.version>
     <javadoc.plugin.version>2.10.3</javadoc.plugin.version>
     <surefire.plugin.version>2.15</surefire.plugin.version>
@@ -579,6 +580,13 @@
           <groupId>com.webcohesion.enunciate</groupId>
           <artifactId>enunciate-maven-plugin</artifactId>
           <version>${enunciate.plugin.version}</version>
+          <dependencies>
+          <dependency>
+            <groupId>javax.annotation</groupId>
+            <artifactId>javax.annotation-api</artifactId>
+            <version>${javax.api.version}</version>
+          </dependency>
+          </dependencies>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>