You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by sh...@apache.org on 2015/10/26 10:07:47 UTC

[01/45] incubator-kylin git commit: KYLIN-978 defer the deletion of the job working on HBase cluster until segment be merged

Repository: incubator-kylin
Updated Branches:
  refs/heads/master 43f119741 -> 9c9fdfa95


KYLIN-978 defer the deletion of the job working on HBase cluster until segment be merged 


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

Branch: refs/heads/master
Commit: 607aeb4fed2eb6b992352f7a0324c52afdaaf2b5
Parents: faf401a
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 21 09:17:50 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 21 09:18:06 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/job/cube/CubingJobBuilder.java | 29 ++++++++------------
 .../kylin/job/cube/GarbageCollectionStep.java   | 23 ++++------------
 2 files changed, 17 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/607aeb4f/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java b/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
index de75f7d..80c030f 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
@@ -63,11 +63,10 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         final CubingJob result = initialJob(seg, "BUILD");
         final String jobId = result.getId();
         final String cuboidRootPath = getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/cuboid/";
-        final List<String> toDeletePathsOnHadoopCluster = Lists.newArrayList();
-        final List<String> toDeletePathsOnHBaseCluster = Lists.newArrayList();
+        final List<String> toDeletePaths = Lists.newArrayList();
 
         // cubing
-        Pair<AbstractExecutable, AbstractExecutable> twoSteps = addCubingSteps(seg, cuboidRootPath, result, toDeletePathsOnHadoopCluster);
+        Pair<AbstractExecutable, AbstractExecutable> twoSteps = addCubingSteps(seg, cuboidRootPath, result, toDeletePaths);
         String intermediateHiveTableStepId = twoSteps.getFirst().getId();
         String baseCuboidStepId = twoSteps.getSecond().getId();
 
@@ -79,8 +78,7 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
 
         final CubeJoinedFlatTableDesc intermediateTableDesc = new CubeJoinedFlatTableDesc(seg.getCubeDesc(), seg);
         final String hiveIntermediateTable = this.getIntermediateHiveTableName(intermediateTableDesc, jobId);
-        toDeletePathsOnHBaseCluster.add(getJobWorkingDir(jobId));
-        result.addTask(createGarbageCollectionStep(seg, null, hiveIntermediateTable, toDeletePathsOnHadoopCluster, toDeletePathsOnHBaseCluster));
+        result.addTask(createGarbageCollectionStep(seg, null, hiveIntermediateTable, toDeletePaths));
 
         return result;
     }
@@ -96,11 +94,10 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         List<String> mergingSegmentIds = Lists.newArrayList();
         List<String> mergingCuboidPaths = Lists.newArrayList();
         List<String> mergingHTables = Lists.newArrayList();
-        final List<String> toDeletePathsOnHadoopCluster = Lists.newArrayList();
-        final List<String> toDeletePathsOnHBaseCluster = Lists.newArrayList();
+        final List<String> toDeletePaths = Lists.newArrayList();
 
         // cubing the incremental segment
-        Pair<AbstractExecutable, AbstractExecutable> twoSteps = addCubingSteps(appendSegment, appendRootPath, result, toDeletePathsOnHadoopCluster);
+        Pair<AbstractExecutable, AbstractExecutable> twoSteps = addCubingSteps(appendSegment, appendRootPath, result, toDeletePaths);
         final String intermediateHiveTableStepId = twoSteps.getFirst().getId();
         final String baseCuboidStepId = twoSteps.getSecond().getId();
 
@@ -118,7 +115,7 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
             } else {
                 mergingCuboidPaths.add(getPathToMerge(merging));
             }
-            toDeletePathsOnHadoopCluster.add(getJobWorkingDir(merging.getLastBuildJobID()));
+            toDeletePaths.add(getJobWorkingDir(merging.getLastBuildJobID()));
         }
 
         // merge cuboid
@@ -129,8 +126,7 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
 
         // update cube info
         result.addTask(createUpdateCubeInfoAfterMergeStep(mergeSegment, mergingSegmentIds, convertCuboidToHfileStep.getId(), jobId));
-        toDeletePathsOnHBaseCluster.add(getJobWorkingDir(jobId));
-        result.addTask(createGarbageCollectionStep(mergeSegment, mergingHTables, null, toDeletePathsOnHadoopCluster, toDeletePathsOnHBaseCluster));
+        result.addTask(createGarbageCollectionStep(mergeSegment, mergingHTables, null, toDeletePaths));
 
         return result;
     }
@@ -147,14 +143,13 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         List<String> mergingSegmentIds = Lists.newArrayList();
         List<String> mergingCuboidPaths = Lists.newArrayList();
         List<String> mergingHTables = Lists.newArrayList();
-        final List<String> toDeletePathsOnHadoopCluster = Lists.newArrayList();
-        final List<String> toDeletePathsOnHBaseCluster = Lists.newArrayList();
+        final List<String> toDeletePaths = Lists.newArrayList();
 
         for (CubeSegment merging : mergingSegments) {
             mergingSegmentIds.add(merging.getUuid());
             mergingCuboidPaths.add(getPathToMerge(merging));
             mergingHTables.add(merging.getStorageLocationIdentifier());
-            toDeletePathsOnHadoopCluster.add(getJobWorkingDir(merging.getLastBuildJobID()));
+            toDeletePaths.add(getJobWorkingDir(merging.getLastBuildJobID()));
         }
 
         // merge cuboid
@@ -165,8 +160,7 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
 
         // update cube info
         result.addTask(createUpdateCubeInfoAfterMergeStep(seg, mergingSegmentIds, convertCuboidToHfileStep.getId(), jobId));
-        toDeletePathsOnHBaseCluster.add(getJobWorkingDir(jobId));
-        result.addTask(createGarbageCollectionStep(seg, mergingHTables, null, toDeletePathsOnHadoopCluster, toDeletePathsOnHBaseCluster));
+        result.addTask(createGarbageCollectionStep(seg, mergingHTables, null, toDeletePaths));
         return result;
     }
 
@@ -470,13 +464,12 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         return result;
     }
 
-    private GarbageCollectionStep createGarbageCollectionStep(CubeSegment seg, List<String> oldHtables, String hiveIntermediateTable, List<String> oldHdsfPaths, List<String> oldHdfsPathsOnHBaseCluster) {
+    private GarbageCollectionStep createGarbageCollectionStep(CubeSegment seg, List<String> oldHtables, String hiveIntermediateTable, List<String> oldHdsfPaths) {
         GarbageCollectionStep result = new GarbageCollectionStep();
         result.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION);
         result.setOldHTables(oldHtables);
         result.setOldHiveTable(hiveIntermediateTable);
         result.setOldHdfsPaths(oldHdsfPaths);
-        result.setOldHdfsPathsOnHBaseCluster(oldHdfsPathsOnHBaseCluster);
         return result;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/607aeb4f/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
index 641454c..f2f1fc0 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
@@ -55,8 +55,6 @@ public class GarbageCollectionStep extends AbstractExecutable {
 
     private static final String OLD_HDFS_PATHS = "oldHdfsPaths";
 
-    private static final String OLD_HDFS_PATHS_ON_HBASE_CLUSTER = "oldHdfsPathsOnHBaseCluster";
-
     private static final Logger logger = LoggerFactory.getLogger(GarbageCollectionStep.class);
 
     private StringBuffer output;
@@ -73,7 +71,6 @@ public class GarbageCollectionStep extends AbstractExecutable {
             dropHBaseTable(context);
             dropHiveTable(context);
             dropHdfsPath(context);
-            dropHdfsPathOnHBaseCluster(context);
         } catch (IOException e) {
             logger.error("job:" + getId() + " execute finished with exception", e);
             output.append("\n").append(e.getLocalizedMessage());
@@ -160,12 +157,12 @@ public class GarbageCollectionStep extends AbstractExecutable {
         List<String> oldHdfsPaths = this.getOldHdfsPaths();
         FileSystem fileSystem = FileSystem.get(HadoopUtil.getCurrentConfiguration());
         dropHdfsPathOnCluster(oldHdfsPaths, fileSystem);
-    }
-
-    private void dropHdfsPathOnHBaseCluster(ExecutableContext context) throws IOException {
-        List<String> oldHdfsPaths = this.getOldHdfsPathsOnHBaseCluster();
-        FileSystem fileSystem = FileSystem.get(HadoopUtil.getCurrentHBaseConfiguration());
-        dropHdfsPathOnCluster(oldHdfsPaths, fileSystem);
+        
+        if (StringUtils.isNotEmpty(context.getConfig().getHBaseClusterFs())) {
+            fileSystem = FileSystem.get(HadoopUtil.getCurrentHBaseConfiguration());
+            dropHdfsPathOnCluster(oldHdfsPaths, fileSystem);
+        }
+        
     }
 
     public void setOldHTables(List<String> tables) {
@@ -184,14 +181,6 @@ public class GarbageCollectionStep extends AbstractExecutable {
         return getArrayParam(OLD_HDFS_PATHS);
     }
 
-    public void setOldHdfsPathsOnHBaseCluster(List<String> paths) {
-        setArrayParam(OLD_HDFS_PATHS_ON_HBASE_CLUSTER, paths);
-    }
-
-    private List<String> getOldHdfsPathsOnHBaseCluster() {
-        return getArrayParam(OLD_HDFS_PATHS_ON_HBASE_CLUSTER);
-    }
-
     private void setArrayParam(String paramKey, List<String> paramValues) {
         setParam(paramKey, StringUtils.join(paramValues, ","));
     }


[31/45] incubator-kylin git commit: KYLIN-1034 Faster bitmap indexes with Roaring bitmaps

Posted by sh...@apache.org.
KYLIN-1034 Faster bitmap indexes with Roaring bitmaps

Close apache/incubator-kylin#12

Signed-off-by: Yang Li <li...@apache.org>


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

Branch: refs/heads/master
Commit: f00c838e6117933e725c3e69f0f30a908541b8a8
Parents: 18610e5
Author: Daniel Lemire <le...@gmail.com>
Authored: Tue Oct 6 18:53:50 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Tue Oct 6 18:54:59 2015 +0800

----------------------------------------------------------------------
 cube/pom.xml                                    |   6 +-
 invertedindex/pom.xml                           |   8 +-
 .../invertedindex/index/BitMapContainer.java    | 504 ++++++++++---------
 .../index/ColumnValueContainer.java             |  86 ++--
 .../index/CompressedValueContainer.java         | 358 ++++++-------
 .../apache/kylin/invertedindex/index/Slice.java |  18 +-
 pom.xml                                         |  10 +-
 .../storage/filter/BitMapFilterEvaluator.java   | 486 +++++++++---------
 .../hbase/coprocessor/endpoint/IIEndpoint.java  |   6 +-
 .../endpoint/SliceBitMapProvider.java           |   4 +-
 .../filter/BitMapFilterEvaluatorTest.java       | 478 +++++++++---------
 11 files changed, 1014 insertions(+), 950 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f00c838e/cube/pom.xml
----------------------------------------------------------------------
diff --git a/cube/pom.xml b/cube/pom.xml
index 2e8ffb0..e5c1f91 100644
--- a/cube/pom.xml
+++ b/cube/pom.xml
@@ -89,9 +89,13 @@
             <artifactId>compress-lzf</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.roaringbitmap</groupId>
+            <artifactId>RoaringBitmap</artifactId>
+        </dependency>
+        <!--<dependency>
             <groupId>com.n3twork.druid</groupId>
             <artifactId>extendedset</artifactId>
-        </dependency>
+        </dependency>-->
 
         <!-- Env & Test -->
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f00c838e/invertedindex/pom.xml
----------------------------------------------------------------------
diff --git a/invertedindex/pom.xml b/invertedindex/pom.xml
index 5d1cf6d..0eaaf1e 100644
--- a/invertedindex/pom.xml
+++ b/invertedindex/pom.xml
@@ -87,9 +87,13 @@
             <artifactId>compress-lzf</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.roaringbitmap</groupId>
+            <artifactId>RoaringBitmap</artifactId>
+        </dependency>
+        <!--<dependency>
             <groupId>com.n3twork.druid</groupId>
             <artifactId>extendedset</artifactId>
-        </dependency>
+        </dependency>-->
 
         <!-- Env & Test -->
 
@@ -158,4 +162,4 @@
         </dependency>
     </dependencies>
 
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f00c838e/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BitMapContainer.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BitMapContainer.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BitMapContainer.java
index 2345261..2594936 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BitMapContainer.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/BitMapContainer.java
@@ -1,226 +1,278 @@
-/*
- * 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.kylin.invertedindex.index;
-
-import java.nio.ByteBuffer;
-import java.nio.IntBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.dict.Dictionary;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-/**
- * @author yangli9
- */
-public class BitMapContainer implements ColumnValueContainer {
-
-    int valueLen;
-    int nValues;
-    int size;
-    ConciseSet[] sets;
-    boolean closedForChange;
-
-    transient byte[] temp;
-
-    public BitMapContainer(TableRecordInfoDigest digest, int col) {
-        this.valueLen = digest.length(col);
-        this.size = 0;
-        this.nValues = digest.getMaxID(col) + 1;
-        this.sets = null;
-        this.closedForChange = false;
-
-        this.temp = new byte[valueLen];
-    }
-
-    @Override
-    public void append(ImmutableBytesWritable valueBytes) {
-        int value = BytesUtil.readUnsigned(valueBytes.get(), valueBytes.getOffset(), valueLen);
-        append(value);
-    }
-
-    public void append(int value) {
-        checkUpdateMode();
-        if (value == Dictionary.NULL_ID[valueLen]) {
-            value = nValues; // set[nValues] holds NULL
-        }
-        sets[value].add(size);
-        size++;
-    }
-
-    @Override
-    public void getValueAt(int i, ImmutableBytesWritable valueBytes) {
-        int value = getValueIntAt(i);
-        BytesUtil.writeUnsigned(value, temp, 0, valueLen);
-        valueBytes.set(temp, 0, valueLen);
-    }
-
-    @Override
-    public ConciseSet getBitMap(Integer startId, Integer endId) {
-        if (startId == null && endId == null) {
-            return sets[this.nValues];
-        }
-
-        int start = 0;
-        int end = this.nValues - 1;
-        if (startId != null) {
-            start = startId;
-        }
-        if (endId != null) {
-            end = endId;
-        }
-
-        ConciseSet ret = new ConciseSet();
-        for (int i = start; i <= end; ++i) {
-            ConciseSet temp = getBitMap(i);
-            ret.addAll(temp);
-        }
-        return ret;
-    }
-
-    private ConciseSet getBitMap(int valueId) {
-        if (valueId >= 0 && valueId <= getMaxValueId())
-            return sets[valueId];
-        else
-            return sets[this.nValues];
-    }
-
-    @Override
-    public int getMaxValueId() {
-        return this.nValues - 1;
-    }
-
-    public int getValueIntAt(int i) {
-        for (int v = 0; v < nValues; v++) {
-            if (sets[v].contains(i)) {
-                return v;
-            }
-        }
-        // if v is not in [0..nValues-1], then it must be nValue (NULL)
-        return Dictionary.NULL_ID[valueLen];
-    }
-
-    private void checkUpdateMode() {
-        if (isClosedForChange()) {
-            throw new IllegalStateException();
-        }
-        if (sets == null) {
-            sets = new ConciseSet[nValues + 1];
-            for (int i = 0; i <= nValues; i++) {
-                sets[i] = new ConciseSet();
-            }
-        }
-    }
-
-    private boolean isClosedForChange() {
-        return closedForChange;
-    }
-
-    @Override
-    public void closeForChange() {
-        closedForChange = true;
-    }
-
-    @Override
-    public int getSize() {
-        return size;
-    }
-
-    public List<ImmutableBytesWritable> toBytes() {
-        if (isClosedForChange() == false)
-            closeForChange();
-
-        List<ImmutableBytesWritable> r = new ArrayList<ImmutableBytesWritable>(nValues + 1);
-        for (int i = 0; i <= nValues; i++) {
-            r.add(setToBytes(sets[i]));
-        }
-        return r;
-    }
-
-    public void fromBytes(List<ImmutableBytesWritable> bytes) {
-        assert nValues + 1 == bytes.size();
-        sets = new ConciseSet[nValues + 1];
-        size = 0;
-        for (int i = 0; i <= nValues; i++) {
-            sets[i] = bytesToSet(bytes.get(i));
-            size += sets[i].size();
-        }
-        closedForChange = true;
-    }
-
-    private ImmutableBytesWritable setToBytes(ConciseSet set) {
-        byte[] array;
-        if (set.isEmpty()) // ConciseSet.toByteBuffer() throws exception when
-                           // set is empty
-            array = BytesUtil.EMPTY_BYTE_ARRAY;
-        else
-            array = set.toByteBuffer().array();
-        return new ImmutableBytesWritable(array);
-    }
-
-    private ConciseSet bytesToSet(ImmutableBytesWritable bytes) {
-        if (bytes.get() == null || bytes.getLength() == 0) {
-            return new ConciseSet();
-        } else {
-            IntBuffer intBuffer = ByteBuffer.wrap(bytes.get(), bytes.getOffset(), bytes.getLength()).asIntBuffer();
-            int[] words = new int[intBuffer.capacity()];
-            intBuffer.get(words);
-            return new ConciseSet(words, false);
-        }
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + (closedForChange ? 1231 : 1237);
-        result = prime * result + nValues;
-        result = prime * result + Arrays.hashCode(sets);
-        result = prime * result + size;
-        result = prime * result + valueLen;
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        BitMapContainer other = (BitMapContainer) obj;
-        if (closedForChange != other.closedForChange)
-            return false;
-        if (nValues != other.nValues)
-            return false;
-        if (!Arrays.equals(sets, other.sets))
-            return false;
-        if (size != other.size)
-            return false;
-        if (valueLen != other.valueLen)
-            return false;
-        return true;
-    }
-
-}
+/*
+ * 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.kylin.invertedindex.index;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.dict.Dictionary;
+import org.roaringbitmap.RoaringBitmap;
+
+/**
+ * @author yangli9
+ */
+public class BitMapContainer implements ColumnValueContainer {
+
+    int valueLen;
+    int nValues;
+    int size;
+    RoaringBitmap[] sets;
+    boolean closedForChange;
+
+    transient byte[] temp;
+
+    public BitMapContainer(TableRecordInfoDigest digest, int col) {
+        this.valueLen = digest.length(col);
+        this.size = 0;
+        this.nValues = digest.getMaxID(col) + 1;
+        this.sets = null;
+        this.closedForChange = false;
+
+        this.temp = new byte[valueLen];
+    }
+
+    @Override
+    public void append(ImmutableBytesWritable valueBytes) {
+        int value = BytesUtil.readUnsigned(valueBytes.get(), valueBytes.getOffset(), valueLen);
+        append(value);
+    }
+
+    public void append(int value) {
+        checkUpdateMode();
+        if (value == Dictionary.NULL_ID[valueLen]) {
+            value = nValues; // set[nValues] holds NULL
+        }
+        sets[value].add(size);
+        size++;
+    }
+
+    @Override
+    public void getValueAt(int i, ImmutableBytesWritable valueBytes) {
+        int value = getValueIntAt(i);
+        BytesUtil.writeUnsigned(value, temp, 0, valueLen);
+        valueBytes.set(temp, 0, valueLen);
+    }
+
+    @Override
+    public RoaringBitmap getBitMap(Integer startId, Integer endId) {
+        if (startId == null && endId == null) {
+            return sets[this.nValues];
+        }
+
+        int start = 0;
+        int end = this.nValues - 1;
+        if (startId != null) {
+            start = startId;
+        }
+        if (endId != null) {
+            end = endId;
+        }
+
+        return RoaringBitmap.or(Arrays.copyOfRange(sets, start, end + 1));
+    }
+
+    @SuppressWarnings("unused")
+    private RoaringBitmap getBitMap(int valueId) {
+        if (valueId >= 0 && valueId <= getMaxValueId())
+            return sets[valueId];
+        else
+            return sets[this.nValues];
+    }
+
+    @Override
+    public int getMaxValueId() {
+        return this.nValues - 1;
+    }
+
+    public int getValueIntAt(int i) {
+        for (int v = 0; v < nValues; v++) {
+            if (sets[v].contains(i)) {
+                return v;
+            }
+        }
+        // if v is not in [0..nValues-1], then it must be nValue (NULL)
+        return Dictionary.NULL_ID[valueLen];
+    }
+
+    private void checkUpdateMode() {
+        if (isClosedForChange()) {
+            throw new IllegalStateException();
+        }
+        if (sets == null) {
+            sets = new RoaringBitmap[nValues + 1];
+            for (int i = 0; i <= nValues; i++) {
+                sets[i] = new RoaringBitmap();
+            }
+        }
+    }
+
+    private boolean isClosedForChange() {
+        return closedForChange;
+    }
+
+    @Override
+    public void closeForChange() {
+        closedForChange = true;
+    }
+
+    @Override
+    public int getSize() {
+        return size;
+    }
+
+    public List<ImmutableBytesWritable> toBytes() {
+        if (isClosedForChange() == false)
+            closeForChange();
+
+        List<ImmutableBytesWritable> r = new ArrayList<ImmutableBytesWritable>(nValues + 1);
+        for (int i = 0; i <= nValues; i++) {
+            r.add(setToBytes(sets[i]));
+        }
+        return r;
+    }
+
+    public void fromBytes(List<ImmutableBytesWritable> bytes) {
+        assert nValues + 1 == bytes.size();
+        sets = new RoaringBitmap[nValues + 1];
+        size = 0;
+        for (int i = 0; i <= nValues; i++) {
+            sets[i] = bytesToSet(bytes.get(i));
+            size += sets[i].getCardinality();
+        }
+        closedForChange = true;
+    }
+
+    private ImmutableBytesWritable setToBytes(RoaringBitmap set) {
+        // Serializing a bitmap to a byte array can be expected to be expensive, this should not be commonly done.
+        // If the purpose is to save the data to disk or to a network, then a direct serialization would be
+        // far more efficient. If the purpose is to enforce immutability, it is an expensive way to do it.
+        set.runOptimize(); //to improve compression
+        final byte[] array = new byte[set.serializedSizeInBytes()];
+        try {
+            set.serialize(new java.io.DataOutputStream(new java.io.OutputStream() {
+                int c = 0;
+
+                @Override
+                public void close() {
+                }
+
+                @Override
+                public void flush() {
+                }
+
+                @Override
+                public void write(int b) {
+                    array[c++] = (byte) b;
+                }
+
+                @Override
+                public void write(byte[] b) {
+                    write(b, 0, b.length);
+                }
+
+                @Override
+                public void write(byte[] b, int off, int l) {
+                    System.arraycopy(b, off, array, c, l);
+                    c += l;
+                }
+            }));
+        } catch (IOException ioe) {
+            // should never happen because we write to a byte array
+            throw new RuntimeException("unexpected error while serializing to a byte array");
+        }
+
+        return new ImmutableBytesWritable(array);
+    }
+
+    private RoaringBitmap bytesToSet(final ImmutableBytesWritable bytes) {
+        // converting a byte array to a bitmap can be expected to be expensive, hopefully this is not a common operation!
+        RoaringBitmap set = new RoaringBitmap();
+        if ((bytes.get() != null) && (bytes.getLength() > 0)) {
+            // here we could use an ImmutableRoaringBitmap and just "map" it.
+            // instead, we do a full deserialization
+            // Note: we deserializing a Roaring bitmap, there is no need to know the length, the format is self-describing
+            try {
+                set.deserialize(new java.io.DataInputStream(new java.io.InputStream() {
+                    byte[] array = bytes.get();
+                    int c = bytes.getOffset();
+
+                    @Override
+                    public int read() {
+                        return array[c++] & 0xff;
+                    }
+
+                    @Override
+                    public int read(byte b[]) {
+                        return read(b, 0, b.length);
+                    }
+
+                    @Override
+                    public int read(byte[] b, int off, int l) {
+                        System.arraycopy(array, c, b, off, l);
+                        c += l;
+                        return l;
+                    }
+                }));
+            } catch (IOException ioe) {
+                // should never happen because we read from a byte array
+                throw new RuntimeException("unexpected error while deserializing from a byte array");
+            }
+        }
+        return set;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + (closedForChange ? 1231 : 1237);
+        result = prime * result + nValues;
+        result = prime * result + Arrays.hashCode(sets);
+        result = prime * result + size;
+        result = prime * result + valueLen;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        BitMapContainer other = (BitMapContainer) obj;
+        if (closedForChange != other.closedForChange)
+            return false;
+        if (nValues != other.nValues)
+            return false;
+        if (!Arrays.equals(sets, other.sets))
+            return false;
+        if (size != other.size)
+            return false;
+        if (valueLen != other.valueLen)
+            return false;
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f00c838e/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ColumnValueContainer.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ColumnValueContainer.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ColumnValueContainer.java
index 77eb422..a5ce1bd 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ColumnValueContainer.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/ColumnValueContainer.java
@@ -1,43 +1,43 @@
-/*
- * 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.kylin.invertedindex.index;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-/**
- * @author yangli9
- */
-public interface ColumnValueContainer {
-
-    void append(ImmutableBytesWritable valueBytes);
-
-    void closeForChange();
-
-    int getSize();
-
-    // works only after closeForChange()
-    void getValueAt(int i, ImmutableBytesWritable valueBytes);
-
-    ConciseSet getBitMap(Integer startId, Integer endId);
-
-    int getMaxValueId();
-
-}
+/*
+ * 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.kylin.invertedindex.index;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import org.roaringbitmap.RoaringBitmap;
+
+/**
+ * @author yangli9
+ */
+public interface ColumnValueContainer {
+
+    void append(ImmutableBytesWritable valueBytes);
+
+    void closeForChange();
+
+    int getSize();
+
+    // works only after closeForChange()
+    void getValueAt(int i, ImmutableBytesWritable valueBytes);
+
+    RoaringBitmap getBitMap(Integer startId, Integer endId);
+
+    int getMaxValueId();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f00c838e/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java
index e301e8d..c58261d 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/CompressedValueContainer.java
@@ -1,179 +1,179 @@
-/*
- * 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.kylin.invertedindex.index;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.dict.Dictionary;
-
-import com.ning.compress.lzf.LZFDecoder;
-import com.ning.compress.lzf.LZFEncoder;
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-/**
- * @author yangli9
- */
-public class CompressedValueContainer implements ColumnValueContainer {
-    int valueLen;
-    int nValues;
-    int cap;
-    int size;
-    byte[] uncompressed;
-    byte[] compressed;
-
-    public CompressedValueContainer(TableRecordInfoDigest digest, int col, int cap) {
-        this.valueLen = digest.length(col);
-        this.nValues = digest.getMaxID(col) + 1;
-        this.cap = cap;
-        this.size = 0;
-        this.uncompressed = null;
-        this.compressed = null;
-    }
-
-    @Override
-    public void append(ImmutableBytesWritable valueBytes) {
-        checkUpdateMode();
-        System.arraycopy(valueBytes.get(), valueBytes.getOffset(), uncompressed, valueLen * size, valueLen);
-        size++;
-    }
-
-    @Override
-    public void getValueAt(int i, ImmutableBytesWritable valueBytes) {
-        valueBytes.set(uncompressed, valueLen * i, valueLen);
-    }
-
-    @Override
-    public ConciseSet getBitMap(Integer startId, Integer endId) {
-        ConciseSet ret = new ConciseSet();
-        int nullId = Dictionary.NULL_ID[valueLen];
-
-        if (startId == null && endId == null) {
-            //entry for getting null values 
-            for (int i = 0; i < size; ++i) {
-                int valueID = BytesUtil.readUnsigned(uncompressed, i * valueLen, valueLen);
-                if (nullId == valueID) {
-                    ret.add(i);
-                }
-            }
-            return ret;
-        }
-
-        //normal values
-        for (int i = 0; i < size; ++i) {
-            int valueID = BytesUtil.readUnsigned(uncompressed, i * valueLen, valueLen);
-            if (valueID == nullId) {
-                continue;
-            }
-
-            if (startId != null && valueID < startId) {
-                continue;
-            }
-
-            if (endId != null && valueID > endId) {
-                continue;
-            }
-
-            ret.add(i);
-        }
-        return ret;
-
-    }
-
-    @Override
-    public int getMaxValueId() {
-        return nValues - 1;
-    }
-
-    private void checkUpdateMode() {
-        if (isClosedForChange()) {
-            throw new IllegalArgumentException();
-        }
-        if (uncompressed == null) {
-            uncompressed = new byte[valueLen * cap];
-        }
-    }
-
-    private boolean isClosedForChange() {
-        return compressed != null;
-    }
-
-    @Override
-    public void closeForChange() {
-        checkUpdateMode();
-        try {
-            compressed = LZFEncoder.encode(uncompressed, 0, valueLen * size);
-        } catch (Exception e) {
-            throw new RuntimeException("LZF encode failure", e);
-        }
-    }
-
-    @Override
-    public int getSize() {
-        return size;
-    }
-
-    public ImmutableBytesWritable toBytes() {
-        if (isClosedForChange() == false)
-            closeForChange();
-        return new ImmutableBytesWritable(compressed);
-    }
-
-    public void fromBytes(ImmutableBytesWritable bytes) {
-        try {
-            uncompressed = LZFDecoder.decode(bytes.get(), bytes.getOffset(), bytes.getLength());
-        } catch (IOException e) {
-            throw new RuntimeException("LZF decode failure", e);
-        }
-        size = cap = uncompressed.length / valueLen;
-        compressed = BytesUtil.EMPTY_BYTE_ARRAY; // mark closed
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + size;
-        result = prime * result + valueLen;
-        result = prime * result + Arrays.hashCode(uncompressed);
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        CompressedValueContainer other = (CompressedValueContainer) obj;
-        if (size != other.size)
-            return false;
-        if (valueLen != other.valueLen)
-            return false;
-        if (!Bytes.equals(uncompressed, 0, size * valueLen, uncompressed, 0, size * valueLen))
-            return false;
-        return true;
-    }
-
-}
+/*
+ * 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.kylin.invertedindex.index;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.dict.Dictionary;
+
+import com.ning.compress.lzf.LZFDecoder;
+import com.ning.compress.lzf.LZFEncoder;
+import org.roaringbitmap.RoaringBitmap;
+
+/**
+ * @author yangli9
+ */
+public class CompressedValueContainer implements ColumnValueContainer {
+    int valueLen;
+    int nValues;
+    int cap;
+    int size;
+    byte[] uncompressed;
+    byte[] compressed;
+
+    public CompressedValueContainer(TableRecordInfoDigest digest, int col, int cap) {
+        this.valueLen = digest.length(col);
+        this.nValues = digest.getMaxID(col) + 1;
+        this.cap = cap;
+        this.size = 0;
+        this.uncompressed = null;
+        this.compressed = null;
+    }
+
+    @Override
+    public void append(ImmutableBytesWritable valueBytes) {
+        checkUpdateMode();
+        System.arraycopy(valueBytes.get(), valueBytes.getOffset(), uncompressed, valueLen * size, valueLen);
+        size++;
+    }
+
+    @Override
+    public void getValueAt(int i, ImmutableBytesWritable valueBytes) {
+        valueBytes.set(uncompressed, valueLen * i, valueLen);
+    }
+
+    @Override
+    public RoaringBitmap getBitMap(Integer startId, Integer endId) {
+        RoaringBitmap ret = new RoaringBitmap();
+        int nullId = Dictionary.NULL_ID[valueLen];
+
+        if (startId == null && endId == null) {
+            //entry for getting null values 
+            for (int i = 0; i < size; ++i) {
+                int valueID = BytesUtil.readUnsigned(uncompressed, i * valueLen, valueLen);
+                if (nullId == valueID) {
+                    ret.add(i);
+                }
+            }
+            return ret;
+        }
+
+        //normal values
+        for (int i = 0; i < size; ++i) {
+            int valueID = BytesUtil.readUnsigned(uncompressed, i * valueLen, valueLen);
+            if (valueID == nullId) {
+                continue;
+            }
+
+            if (startId != null && valueID < startId) {
+                continue;
+            }
+
+            if (endId != null && valueID > endId) {
+                continue;
+            }
+
+            ret.add(i);
+        }
+        return ret;
+
+    }
+
+    @Override
+    public int getMaxValueId() {
+        return nValues - 1;
+    }
+
+    private void checkUpdateMode() {
+        if (isClosedForChange()) {
+            throw new IllegalArgumentException();
+        }
+        if (uncompressed == null) {
+            uncompressed = new byte[valueLen * cap];
+        }
+    }
+
+    private boolean isClosedForChange() {
+        return compressed != null;
+    }
+
+    @Override
+    public void closeForChange() {
+        checkUpdateMode();
+        try {
+            compressed = LZFEncoder.encode(uncompressed, 0, valueLen * size);
+        } catch (Exception e) {
+            throw new RuntimeException("LZF encode failure", e);
+        }
+    }
+
+    @Override
+    public int getSize() {
+        return size;
+    }
+
+    public ImmutableBytesWritable toBytes() {
+        if (isClosedForChange() == false)
+            closeForChange();
+        return new ImmutableBytesWritable(compressed);
+    }
+
+    public void fromBytes(ImmutableBytesWritable bytes) {
+        try {
+            uncompressed = LZFDecoder.decode(bytes.get(), bytes.getOffset(), bytes.getLength());
+        } catch (IOException e) {
+            throw new RuntimeException("LZF decode failure", e);
+        }
+        size = cap = uncompressed.length / valueLen;
+        compressed = BytesUtil.EMPTY_BYTE_ARRAY; // mark closed
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + size;
+        result = prime * result + valueLen;
+        result = prime * result + Arrays.hashCode(uncompressed);
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        CompressedValueContainer other = (CompressedValueContainer) obj;
+        if (size != other.size)
+            return false;
+        if (valueLen != other.valueLen)
+            return false;
+        if (!Bytes.equals(uncompressed, 0, size * valueLen, uncompressed, 0, size * valueLen))
+            return false;
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f00c838e/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java
index cccd503..2a53864 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/index/Slice.java
@@ -22,7 +22,8 @@ import java.util.Iterator;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
+import org.roaringbitmap.RoaringBitmap;
+import org.roaringbitmap.IntIterator;
 
 /**
  * Within a partition (per timestampGranularity), records are further sliced
@@ -75,7 +76,7 @@ public class Slice implements Iterable<RawTableRecord>, Comparable<Slice> {
         return containers[col];
     }
 
-    public Iterator<RawTableRecord> iterateWithBitmap(final ConciseSet resultBitMap) {
+    public Iterator<RawTableRecord> iterateWithBitmap(final RoaringBitmap resultBitMap) {
         if (resultBitMap == null) {
             return this.iterator();
         } else {
@@ -83,27 +84,20 @@ public class Slice implements Iterable<RawTableRecord>, Comparable<Slice> {
             final ImmutableBytesWritable temp = new ImmutableBytesWritable();
 
             return new Iterator<RawTableRecord>() {
-                int i = 0;
-                int iteratedCount = 0;
-                int resultSize = resultBitMap.size();
+                IntIterator iter = resultBitMap.getIntIterator();
 
                 @Override
                 public boolean hasNext() {
-                    return iteratedCount < resultSize;
+                    return iter.hasNext();
                 }
 
                 @Override
                 public RawTableRecord next() {
-                    while (!resultBitMap.contains(i)) {
-                        i++;
-                    }
+                    int i = iter.next();
                     for (int col = 0; col < nColumns; col++) {
                         containers[col].getValueAt(i, temp);
                         rec.setValueBytes(col, temp);
                     }
-                    iteratedCount++;
-                    i++;
-
                     return rec;
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f00c838e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b1df6e3..db9ab91 100644
--- a/pom.xml
+++ b/pom.xml
@@ -77,7 +77,8 @@
         <xerces.version>2.9.1</xerces.version>
         <xalan.version>2.7.1</xalan.version>
         <compress-lzf.version>1.0.3</compress-lzf.version>
-        <extendedset.version>1.3.4</extendedset.version>
+        <!--<extendedset.version>1.3.4</extendedset.version>-->
+        <roaring.version>[0.5.4,)</roaring.version>
         <jetty.version>9.2.7.v20150116</jetty.version>
         <opencsv.version>2.3</opencsv.version>
 
@@ -376,10 +377,15 @@
                 <version>${compress-lzf.version}</version>
             </dependency>
             <dependency>
+                <groupId>org.roaringbitmap</groupId>
+                <artifactId>RoaringBitmap</artifactId>
+                <version>${roaring.version}</version>
+            </dependency>
+            <!--<dependency>
                 <groupId>com.n3twork.druid</groupId>
                 <artifactId>extendedset</artifactId>
                 <version>${extendedset.version}</version>
-            </dependency>
+            </dependency>-->
 
             <dependency>
                 <groupId>org.apache.curator</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f00c838e/storage/src/main/java/org/apache/kylin/storage/filter/BitMapFilterEvaluator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/filter/BitMapFilterEvaluator.java b/storage/src/main/java/org/apache/kylin/storage/filter/BitMapFilterEvaluator.java
index 0387abc..49f1c32 100644
--- a/storage/src/main/java/org/apache/kylin/storage/filter/BitMapFilterEvaluator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/filter/BitMapFilterEvaluator.java
@@ -1,242 +1,244 @@
-/*
- * 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.kylin.storage.filter;
-
-import java.util.List;
-
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.LogicalTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-/**
- * @author yangli9
- *
- * Evaluate a group of records against a filter in batch.
- */
-public class BitMapFilterEvaluator {
-
-    /** Provides bitmaps for a record group ranging [0..N-1], where N is the size of the group */
-    public static interface BitMapProvider {
-
-        /** return records whose specified column having specified value */
-        ConciseSet getBitMap(TblColRef col, Integer startId, Integer endId);
-
-        /** return the size of the group */
-        int getRecordCount();
-
-        /** return the max value ID of a column according to dictionary */
-        int getMaxValueId(TblColRef col);
-    }
-
-    BitMapProvider provider;
-
-    public BitMapFilterEvaluator(BitMapProvider bitMapProvider) {
-        this.provider = bitMapProvider;
-    }
-
-    /**
-     * @param filter
-     * @return a set of records that match the filter; or null if filter is null or unable to evaluate
-     */
-    public ConciseSet evaluate(TupleFilter filter) {
-        if (filter == null)
-            return null;
-
-        if (filter instanceof LogicalTupleFilter)
-            return evalLogical((LogicalTupleFilter) filter);
-
-        if (filter instanceof CompareTupleFilter)
-            return evalCompare((CompareTupleFilter) filter);
-
-        return null; // unable to evaluate
-    }
-
-    private ConciseSet evalCompare(CompareTupleFilter filter) {
-        switch (filter.getOperator()) {
-        case ISNULL:
-            return evalCompareIsNull(filter);
-        case ISNOTNULL:
-            return evalCompareIsNotNull(filter);
-        case EQ:
-            return evalCompareEqual(filter);
-        case NEQ:
-            return evalCompareNotEqual(filter);
-        case IN:
-            return evalCompareIn(filter);
-        case NOTIN:
-            return evalCompareNotIn(filter);
-        case LT:
-            return evalCompareLT(filter);
-        case LTE:
-            return evalCompareLTE(filter);
-        case GT:
-            return evalCompareGT(filter);
-        case GTE:
-            return evalCompareGTE(filter);
-        default:
-            throw new IllegalStateException("Unsupported operator " + filter.getOperator());
-        }
-    }
-
-    private ConciseSet evalCompareLT(CompareTupleFilter filter) {
-        int id = Dictionary.stringToDictId(filter.getFirstValue());
-        return collectRange(filter.getColumn(), null, id - 1);
-    }
-
-    private ConciseSet evalCompareLTE(CompareTupleFilter filter) {
-        int id = Dictionary.stringToDictId(filter.getFirstValue());
-        return collectRange(filter.getColumn(), null, id);
-    }
-
-    private ConciseSet evalCompareGT(CompareTupleFilter filter) {
-        int id = Dictionary.stringToDictId(filter.getFirstValue());
-        return collectRange(filter.getColumn(), id + 1, null);
-    }
-
-    private ConciseSet evalCompareGTE(CompareTupleFilter filter) {
-        int id = Dictionary.stringToDictId(filter.getFirstValue());
-        return collectRange(filter.getColumn(), id, null);
-    }
-
-    private ConciseSet collectRange(TblColRef column, Integer startId, Integer endId) {
-        return provider.getBitMap(column, startId, endId);
-    }
-
-    private ConciseSet evalCompareEqual(CompareTupleFilter filter) {
-        int id = Dictionary.stringToDictId(filter.getFirstValue());
-        ConciseSet bitMap = provider.getBitMap(filter.getColumn(), id, id);
-        if (bitMap == null)
-            return null;
-        return bitMap.clone(); // NOTE the clone() to void messing provider's cache
-    }
-
-    private ConciseSet evalCompareNotEqual(CompareTupleFilter filter) {
-        ConciseSet set = evalCompareEqual(filter);
-        not(set);
-        dropNull(set, filter);
-        return set;
-    }
-
-    private ConciseSet evalCompareIn(CompareTupleFilter filter) {
-        ConciseSet set = new ConciseSet();
-        for (String value : filter.getValues()) {
-            int id = Dictionary.stringToDictId(value);
-            ConciseSet bitMap = provider.getBitMap(filter.getColumn(), id, id);
-            if (bitMap == null)
-                return null;
-            set.addAll(bitMap);
-        }
-        return set;
-    }
-
-    private ConciseSet evalCompareNotIn(CompareTupleFilter filter) {
-        ConciseSet set = evalCompareIn(filter);
-        not(set);
-        dropNull(set, filter);
-        return set;
-    }
-
-    private void dropNull(ConciseSet set, CompareTupleFilter filter) {
-        if (set == null)
-            return;
-
-        ConciseSet nullSet = evalCompareIsNull(filter);
-        set.removeAll(nullSet);
-    }
-
-    private ConciseSet evalCompareIsNull(CompareTupleFilter filter) {
-        ConciseSet bitMap = provider.getBitMap(filter.getColumn(), null, null);
-        if (bitMap == null)
-            return null;
-        return bitMap.clone(); // NOTE the clone() to void messing provider's cache
-    }
-
-    private ConciseSet evalCompareIsNotNull(CompareTupleFilter filter) {
-        ConciseSet set = evalCompareIsNull(filter);
-        not(set);
-        return set;
-    }
-
-    private ConciseSet evalLogical(LogicalTupleFilter filter) {
-        List<? extends TupleFilter> children = filter.getChildren();
-
-        switch (filter.getOperator()) {
-        case AND:
-            return evalLogicalAnd(children);
-        case OR:
-            return evalLogicalOr(children);
-        case NOT:
-            return evalLogicalNot(children);
-        default:
-            throw new IllegalStateException("Unsupported operator " + filter.getOperator());
-        }
-    }
-
-    private ConciseSet evalLogicalAnd(List<? extends TupleFilter> children) {
-        ConciseSet set = new ConciseSet();
-        not(set);
-
-        for (TupleFilter c : children) {
-            ConciseSet t = evaluate(c);
-            if (t == null)
-                continue; // because it's AND
-
-            set.retainAll(t);
-        }
-        return set;
-    }
-
-    private ConciseSet evalLogicalOr(List<? extends TupleFilter> children) {
-        ConciseSet set = new ConciseSet();
-
-        for (TupleFilter c : children) {
-            ConciseSet t = evaluate(c);
-            if (t == null)
-                return null; // because it's OR
-
-            set.addAll(t);
-        }
-        return set;
-    }
-
-    private ConciseSet evalLogicalNot(List<? extends TupleFilter> children) {
-        ConciseSet set = evaluate(children.get(0));
-        not(set);
-        return set;
-    }
-
-    private void not(ConciseSet set) {
-        if (set == null)
-            return;
-
-        set.add(provider.getRecordCount());
-        set.complement();
-    }
-
-    public static void main(String[] args) {
-        ConciseSet s = new ConciseSet();
-        s.add(5);
-        s.complement();
-        System.out.println(s);
-    }
-}
+/*
+ * 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.kylin.storage.filter;
+
+import java.util.List;
+
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.metadata.filter.CompareTupleFilter;
+import org.apache.kylin.metadata.filter.LogicalTupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import org.roaringbitmap.RoaringBitmap;
+
+/**
+ * @author yangli9
+ *
+ * Evaluate a group of records against a filter in batch.
+ */
+public class BitMapFilterEvaluator {
+
+    /** Provides bitmaps for a record group ranging [0..N-1], where N is the size of the group */
+    public static interface BitMapProvider {
+
+        /** return records whose specified column having specified value */
+        RoaringBitmap getBitMap(TblColRef col, Integer startId, Integer endId);
+
+        /** return the size of the group */
+        int getRecordCount();
+
+        /** return the max value ID of a column according to dictionary */
+        int getMaxValueId(TblColRef col);
+    }
+
+    BitMapProvider provider;
+
+    public BitMapFilterEvaluator(BitMapProvider bitMapProvider) {
+        this.provider = bitMapProvider;
+    }
+
+    /**
+     * @param filter
+     * @return a set of records that match the filter; or null if filter is null or unable to evaluate
+     */
+    public RoaringBitmap evaluate(TupleFilter filter) {
+        if (filter == null)
+            return null;
+
+        if (filter instanceof LogicalTupleFilter)
+            return evalLogical((LogicalTupleFilter) filter);
+
+        if (filter instanceof CompareTupleFilter)
+            return evalCompare((CompareTupleFilter) filter);
+
+        return null; // unable to evaluate
+    }
+
+    private RoaringBitmap evalCompare(CompareTupleFilter filter) {
+        switch (filter.getOperator()) {
+        case ISNULL:
+            return evalCompareIsNull(filter);
+        case ISNOTNULL:
+            return evalCompareIsNotNull(filter);
+        case EQ:
+            return evalCompareEqual(filter);
+        case NEQ:
+            return evalCompareNotEqual(filter);
+        case IN:
+            return evalCompareIn(filter);
+        case NOTIN:
+            return evalCompareNotIn(filter);
+        case LT:
+            return evalCompareLT(filter);
+        case LTE:
+            return evalCompareLTE(filter);
+        case GT:
+            return evalCompareGT(filter);
+        case GTE:
+            return evalCompareGTE(filter);
+        default:
+            throw new IllegalStateException("Unsupported operator " + filter.getOperator());
+        }
+    }
+
+    private RoaringBitmap evalCompareLT(CompareTupleFilter filter) {
+        int id = Dictionary.stringToDictId(filter.getFirstValue());
+        return collectRange(filter.getColumn(), null, id - 1);
+    }
+
+    private RoaringBitmap evalCompareLTE(CompareTupleFilter filter) {
+        int id = Dictionary.stringToDictId(filter.getFirstValue());
+        return collectRange(filter.getColumn(), null, id);
+    }
+
+    private RoaringBitmap evalCompareGT(CompareTupleFilter filter) {
+        int id = Dictionary.stringToDictId(filter.getFirstValue());
+        return collectRange(filter.getColumn(), id + 1, null);
+    }
+
+    private RoaringBitmap evalCompareGTE(CompareTupleFilter filter) {
+        int id = Dictionary.stringToDictId(filter.getFirstValue());
+        return collectRange(filter.getColumn(), id, null);
+    }
+
+    private RoaringBitmap collectRange(TblColRef column, Integer startId, Integer endId) {
+        return provider.getBitMap(column, startId, endId);
+    }
+
+    private RoaringBitmap evalCompareEqual(CompareTupleFilter filter) {
+        int id = Dictionary.stringToDictId(filter.getFirstValue());
+        RoaringBitmap bitMap = provider.getBitMap(filter.getColumn(), id, id);
+        if (bitMap == null)
+            return null;
+        return bitMap.clone(); // NOTE the clone() to void messing provider's cache // If the object is immutable, this is likely wasteful
+    }
+
+    private RoaringBitmap evalCompareNotEqual(CompareTupleFilter filter) {
+        RoaringBitmap set = evalCompareEqual(filter);
+        not(set);
+        dropNull(set, filter);
+        return set;
+    }
+
+    private RoaringBitmap evalCompareIn(CompareTupleFilter filter) {
+        java.util.ArrayList<RoaringBitmap> buffer = new java.util.ArrayList<RoaringBitmap>();
+        // an iterator would be better than an ArrayList, but there is
+        // the convention that says that if one bitmap is null, we return null...
+        for (String value : filter.getValues()) {
+            int id = Dictionary.stringToDictId(value);
+            RoaringBitmap bitMap = provider.getBitMap(filter.getColumn(), id, id);
+            if (bitMap == null)
+                return null;
+            buffer.add(bitMap);
+        }
+        return RoaringBitmap.or(buffer.iterator());
+    }
+
+    private RoaringBitmap evalCompareNotIn(CompareTupleFilter filter) {
+        RoaringBitmap set = evalCompareIn(filter);
+        not(set);
+        dropNull(set, filter);
+        return set;
+    }
+
+    private void dropNull(RoaringBitmap set, CompareTupleFilter filter) {
+        if (set == null)
+            return;
+
+        RoaringBitmap nullSet = evalCompareIsNull(filter);
+        set.andNot(nullSet);
+    }
+
+    private RoaringBitmap evalCompareIsNull(CompareTupleFilter filter) {
+        RoaringBitmap bitMap = provider.getBitMap(filter.getColumn(), null, null);
+        if (bitMap == null)
+            return null;
+        return bitMap.clone(); // NOTE the clone() to void messing provider's cache // If the object is immutable, this is likely wasteful
+    }
+
+    private RoaringBitmap evalCompareIsNotNull(CompareTupleFilter filter) {
+        RoaringBitmap set = evalCompareIsNull(filter);
+        not(set);
+        return set;
+    }
+
+    private RoaringBitmap evalLogical(LogicalTupleFilter filter) {
+        List<? extends TupleFilter> children = filter.getChildren();
+
+        switch (filter.getOperator()) {
+        case AND:
+            return evalLogicalAnd(children);
+        case OR:
+            return evalLogicalOr(children);
+        case NOT:
+            return evalLogicalNot(children);
+        default:
+            throw new IllegalStateException("Unsupported operator " + filter.getOperator());
+        }
+    }
+
+    private RoaringBitmap evalLogicalAnd(List<? extends TupleFilter> children) {
+        int i = 0;
+        RoaringBitmap answer = null;
+        // we identify the first non-null
+        for(; (i < children.size()) && (answer == null); ++i) {
+            RoaringBitmap t = evaluate(children.get(i));
+            if (t == null)
+                continue; // because it's AND // following convention
+            answer = t;
+        }
+        // then we compute the intersections
+        for(; i < children.size(); ++i) {
+            RoaringBitmap t = evaluate(children.get(i));
+            if (t == null)
+                continue; // because it's AND // following convention
+            answer.and(t);
+        }
+        if(answer == null)
+            answer = new RoaringBitmap();
+        return answer;
+    }
+
+    private RoaringBitmap evalLogicalOr(List<? extends TupleFilter> children) {
+        java.util.ArrayList<RoaringBitmap> buffer = new java.util.ArrayList<RoaringBitmap>();
+        // could be done with iterator but there is the rule if that if there is a null, then we need to return null
+        for (TupleFilter c : children) {
+            RoaringBitmap t = evaluate(c);
+            if (t == null)
+                return null; // because it's OR // following convention
+            buffer.add(t);
+        }
+        return RoaringBitmap.or(buffer.iterator());
+    }
+
+    private RoaringBitmap evalLogicalNot(List<? extends TupleFilter> children) {
+        RoaringBitmap set = evaluate(children.get(0));
+        not(set);
+        return set;
+    }
+
+    private void not(RoaringBitmap set) {
+        if (set == null)
+            return;
+        set.flip(0,provider.getRecordCount());
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f00c838e/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
index c20b09b..d55361b 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
@@ -51,7 +51,7 @@ import com.google.protobuf.ByteString;
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
+import org.roaringbitmap.RoaringBitmap;
 
 /**
  * Created by honma on 11/7/14.
@@ -128,7 +128,7 @@ public class IIEndpoint extends IIProtos.RowsService implements Coprocessor, Cop
         EndpointAggregationCache aggCache = new EndpointAggregationCache(aggregators);
         IIProtos.IIResponse.Builder responseBuilder = IIProtos.IIResponse.newBuilder();
         for (Slice slice : slices) {
-            ConciseSet result = null;
+            RoaringBitmap result = null;
             if (filter != null) {
                 result = new BitMapFilterEvaluator(new SliceBitMapProvider(slice, type)).evaluate(filter.getFilter());
             }
@@ -158,7 +158,7 @@ public class IIEndpoint extends IIProtos.RowsService implements Coprocessor, Cop
     private IIProtos.IIResponse getNonAggregatedResponse(Iterable<Slice> slices, CoprocessorFilter filter, CoprocessorRowType type) {
         IIProtos.IIResponse.Builder responseBuilder = IIProtos.IIResponse.newBuilder();
         for (Slice slice : slices) {
-            ConciseSet result = null;
+            RoaringBitmap result = null;
             if (filter != null) {
                 result = new BitMapFilterEvaluator(new SliceBitMapProvider(slice, type)).evaluate(filter.getFilter());
             }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f00c838e/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/SliceBitMapProvider.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/SliceBitMapProvider.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/SliceBitMapProvider.java
index 76adadd..c5bee40 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/SliceBitMapProvider.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/SliceBitMapProvider.java
@@ -23,7 +23,7 @@ import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.storage.filter.BitMapFilterEvaluator;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorRowType;
 
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
+import org.roaringbitmap.RoaringBitmap;
 
 /**
  * Created by Hongbin Ma(Binmahone) on 11/24/14.
@@ -41,7 +41,7 @@ public class SliceBitMapProvider implements BitMapFilterEvaluator.BitMapProvider
     }
 
     @Override
-    public ConciseSet getBitMap(TblColRef col, Integer startId, Integer endId) {
+    public RoaringBitmap getBitMap(TblColRef col, Integer startId, Integer endId) {
         return slice.getColumnValueContainer(type.getColIndexByTblColRef(col)).getBitMap(startId, endId);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f00c838e/storage/src/test/java/org/apache/kylin/storage/filter/BitMapFilterEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/filter/BitMapFilterEvaluatorTest.java b/storage/src/test/java/org/apache/kylin/storage/filter/BitMapFilterEvaluatorTest.java
index ca6a957..5a3041d 100644
--- a/storage/src/test/java/org/apache/kylin/storage/filter/BitMapFilterEvaluatorTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/filter/BitMapFilterEvaluatorTest.java
@@ -1,238 +1,240 @@
-/*
- * 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.kylin.storage.filter;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-
-import org.apache.kylin.dict.Dictionary;
-import org.apache.kylin.metadata.filter.ColumnTupleFilter;
-import org.apache.kylin.metadata.filter.CompareTupleFilter;
-import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.LogicalTupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
-import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.storage.filter.BitMapFilterEvaluator.BitMapProvider;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-import it.uniroma3.mat.extendedset.intset.ConciseSet;
-
-public class BitMapFilterEvaluatorTest {
-
-    static TblColRef colA;
-    static TblColRef colB;
-
-    static {
-        TableDesc table = new TableDesc();
-        table.setName("TABLE");
-        table.setDatabase("DEFAULT");
-
-        ColumnDesc col = new ColumnDesc();
-        col.setTable(table);
-        col.setName("colA");
-        colA = new TblColRef(col);
-
-        col = new ColumnDesc();
-        col.setTable(table);
-        col.setName("colB");
-        colB = new TblColRef(col);
-    }
-
-    static class MockBitMapProivder implements BitMapProvider {
-
-        private static final int MAX_ID = 8;
-        private static final int REC_COUNT = 10;
-
-        @Override
-        public ConciseSet getBitMap(TblColRef col, Integer startId, Integer endId) {
-            if (!col.equals(colA))
-                return null;
-
-            // i-th record has value ID i, and last record has value null
-            if (startId == null && endId == null) {
-                //entry for getting null value
-                ConciseSet s = new ConciseSet();
-                s.add(getRecordCount() - 1);
-                return s;
-            }
-
-            int start = 0;
-            int end = MAX_ID;
-            if (startId != null) {
-                start = startId;
-            }
-            if (endId != null) {
-                end = endId;
-            }
-
-            ConciseSet ret = new ConciseSet();
-            for (int i = start; i <= end; ++i) {
-                ConciseSet temp = getBitMap(col, i);
-                ret.addAll(temp);
-            }
-            return ret;
-        }
-
-        public ConciseSet getBitMap(TblColRef col, int valueId) {
-            if (!col.equals(colA))
-                return null;
-
-            // i-th record has value ID i, and last record has value null
-            ConciseSet bitMap = new ConciseSet();
-            if (valueId < 0 || valueId > getMaxValueId(col)) // null
-                bitMap.add(getRecordCount() - 1);
-            else
-                bitMap.add(valueId);
-
-            return bitMap;
-        }
-
-        @Override
-        public int getRecordCount() {
-            return REC_COUNT;
-        }
-
-        @Override
-        public int getMaxValueId(TblColRef col) {
-            return MAX_ID;
-        }
-    }
-
-    BitMapFilterEvaluator eval = new BitMapFilterEvaluator(new MockBitMapProivder());
-    ArrayList<CompareTupleFilter> basicFilters = Lists.newArrayList();
-    ArrayList<ConciseSet> basicResults = Lists.newArrayList();
-
-    public BitMapFilterEvaluatorTest() {
-        basicFilters.add(compare(colA, FilterOperatorEnum.ISNULL));
-        basicResults.add(set(9));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.ISNOTNULL));
-        basicResults.add(set(0, 1, 2, 3, 4, 5, 6, 7, 8));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.EQ, 0));
-        basicResults.add(set(0));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.NEQ, 0));
-        basicResults.add(set(1, 2, 3, 4, 5, 6, 7, 8));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.IN, 0, 5));
-        basicResults.add(set(0, 5));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.NOTIN, 0, 5));
-        basicResults.add(set(1, 2, 3, 4, 6, 7, 8));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.LT, 3));
-        basicResults.add(set(0, 1, 2));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.LTE, 3));
-        basicResults.add(set(0, 1, 2, 3));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.GT, 3));
-        basicResults.add(set(4, 5, 6, 7, 8));
-
-        basicFilters.add(compare(colA, FilterOperatorEnum.GTE, 3));
-        basicResults.add(set(3, 4, 5, 6, 7, 8));
-    }
-
-    @Test
-    public void testBasics() {
-        for (int i = 0; i < basicFilters.size(); i++) {
-            assertEquals(basicResults.get(i), eval.evaluate(basicFilters.get(i)));
-        }
-    }
-
-    @Test
-    public void testLogicalAnd() {
-        for (int i = 0; i < basicFilters.size(); i++) {
-            for (int j = 0; j < basicFilters.size(); j++) {
-                LogicalTupleFilter f = logical(FilterOperatorEnum.AND, basicFilters.get(i), basicFilters.get(j));
-                ConciseSet r = basicResults.get(i).clone();
-                r.retainAll(basicResults.get(j));
-                assertEquals(r, eval.evaluate(f));
-            }
-        }
-    }
-
-    @Test
-    public void testLogicalOr() {
-        for (int i = 0; i < basicFilters.size(); i++) {
-            for (int j = 0; j < basicFilters.size(); j++) {
-                LogicalTupleFilter f = logical(FilterOperatorEnum.OR, basicFilters.get(i), basicFilters.get(j));
-                ConciseSet r = basicResults.get(i).clone();
-                r.addAll(basicResults.get(j));
-                assertEquals(r, eval.evaluate(f));
-            }
-        }
-    }
-
-    @Test
-    public void testNotEvaluable() {
-        CompareTupleFilter notEvaluable = compare(colB, FilterOperatorEnum.EQ, 0);
-        assertEquals(null, eval.evaluate(notEvaluable));
-
-        LogicalTupleFilter or = logical(FilterOperatorEnum.OR, basicFilters.get(1), notEvaluable);
-        assertEquals(null, eval.evaluate(or));
-
-        LogicalTupleFilter and = logical(FilterOperatorEnum.AND, basicFilters.get(1), notEvaluable);
-        assertEquals(basicResults.get(1), eval.evaluate(and));
-    }
-
-    public static CompareTupleFilter compare(TblColRef col, TupleFilter.FilterOperatorEnum op, int... ids) {
-        CompareTupleFilter filter = new CompareTupleFilter(op);
-        filter.setNullString(idToStr(Dictionary.NULL_ID[1]));
-        filter.addChild(columnFilter(col));
-        for (int i : ids) {
-            filter.addChild(constFilter(i));
-        }
-        return filter;
-    }
-
-    public static LogicalTupleFilter logical(TupleFilter.FilterOperatorEnum op, TupleFilter... filters) {
-        LogicalTupleFilter filter = new LogicalTupleFilter(op);
-        for (TupleFilter f : filters)
-            filter.addChild(f);
-        return filter;
-    }
-
-    public static ColumnTupleFilter columnFilter(TblColRef col) {
-        return new ColumnTupleFilter(col);
-    }
-
-    public static ConstantTupleFilter constFilter(int id) {
-        return new ConstantTupleFilter(idToStr(id));
-    }
-
-    public static ConciseSet set(int... ints) {
-        ConciseSet set = new ConciseSet();
-        for (int i : ints)
-            set.add(i);
-        return set;
-    }
-
-    public static String idToStr(int id) {
-        byte[] bytes = new byte[] { (byte) id };
-        return Dictionary.dictIdToString(bytes, 0, bytes.length);
-    }
-
-}
+/*
+ * 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.kylin.storage.filter;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.metadata.filter.ColumnTupleFilter;
+import org.apache.kylin.metadata.filter.CompareTupleFilter;
+import org.apache.kylin.metadata.filter.ConstantTupleFilter;
+import org.apache.kylin.metadata.filter.LogicalTupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.storage.filter.BitMapFilterEvaluator.BitMapProvider;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+import org.roaringbitmap.RoaringBitmap;
+
+public class BitMapFilterEvaluatorTest {
+
+    static TblColRef colA;
+    static TblColRef colB;
+
+    static {
+        TableDesc table = new TableDesc();
+        table.setName("TABLE");
+        table.setDatabase("DEFAULT");
+
+        ColumnDesc col = new ColumnDesc();
+        col.setTable(table);
+        col.setName("colA");
+        colA = new TblColRef(col);
+
+        col = new ColumnDesc();
+        col.setTable(table);
+        col.setName("colB");
+        colB = new TblColRef(col);
+    }
+
+    static class MockBitMapProivder implements BitMapProvider {
+
+        private static final int MAX_ID = 8;
+        private static final int REC_COUNT = 10;
+
+        @Override
+        public RoaringBitmap getBitMap(final TblColRef col, Integer startId, Integer endId) {
+            if (!col.equals(colA))
+                return null;
+
+            // i-th record has value ID i, and last record has value null
+            if (startId == null && endId == null) {
+                //entry for getting null value
+                return RoaringBitmap.bitmapOf(getRecordCount() - 1);
+            }
+
+            final int start = (startId != null) ? startId : 0;
+            final int end = (endId != null) ? endId : MAX_ID;
+            return RoaringBitmap.or(new Iterator<RoaringBitmap>() {
+                int i = start;
+
+                @Override
+                public boolean hasNext() {
+                    return i <= end;
+                }
+
+                @Override
+                public RoaringBitmap next() {
+                    return getBitMap(col, i++);
+                }
+
+                @Override
+                public void remove() {
+                    throw new UnsupportedOperationException();
+                }
+            });
+        }
+
+        public RoaringBitmap getBitMap(TblColRef col, int valueId) {
+            if (!col.equals(colA))
+                return null;
+
+            // i-th record has value ID i, and last record has value null
+            RoaringBitmap bitMap = new RoaringBitmap();
+            if (valueId < 0 || valueId > getMaxValueId(col)) // null
+                bitMap.add(getRecordCount() - 1);
+            else
+                bitMap.add(valueId);
+
+            return bitMap;
+        }
+
+        @Override
+        public int getRecordCount() {
+            return REC_COUNT;
+        }
+
+        @Override
+        public int getMaxValueId(TblColRef col) {
+            return MAX_ID;
+        }
+    }
+
+    BitMapFilterEvaluator eval = new BitMapFilterEvaluator(new MockBitMapProivder());
+    ArrayList<CompareTupleFilter> basicFilters = Lists.newArrayList();
+    ArrayList<RoaringBitmap> basicResults = Lists.newArrayList();
+
+    public BitMapFilterEvaluatorTest() {
+        basicFilters.add(compare(colA, FilterOperatorEnum.ISNULL));
+        basicResults.add(set(9));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.ISNOTNULL));
+        basicResults.add(set(0, 1, 2, 3, 4, 5, 6, 7, 8));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.EQ, 0));
+        basicResults.add(set(0));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.NEQ, 0));
+        basicResults.add(set(1, 2, 3, 4, 5, 6, 7, 8));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.IN, 0, 5));
+        basicResults.add(set(0, 5));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.NOTIN, 0, 5));
+        basicResults.add(set(1, 2, 3, 4, 6, 7, 8));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.LT, 3));
+        basicResults.add(set(0, 1, 2));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.LTE, 3));
+        basicResults.add(set(0, 1, 2, 3));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.GT, 3));
+        basicResults.add(set(4, 5, 6, 7, 8));
+
+        basicFilters.add(compare(colA, FilterOperatorEnum.GTE, 3));
+        basicResults.add(set(3, 4, 5, 6, 7, 8));
+    }
+
+    @Test
+    public void testBasics() {
+        for (int i = 0; i < basicFilters.size(); i++) {
+            assertEquals(basicResults.get(i), eval.evaluate(basicFilters.get(i)));
+        }
+    }
+
+    @Test
+    public void testLogicalAnd() {
+        for (int i = 0; i < basicFilters.size(); i++) {
+            for (int j = 0; j < basicFilters.size(); j++) {
+                LogicalTupleFilter f = logical(FilterOperatorEnum.AND, basicFilters.get(i), basicFilters.get(j));
+                RoaringBitmap r = basicResults.get(i).clone();
+                r.and(basicResults.get(j));
+                assertEquals(r, eval.evaluate(f));
+            }
+        }
+    }
+
+    @Test
+    public void testLogicalOr() {
+        for (int i = 0; i < basicFilters.size(); i++) {
+            for (int j = 0; j < basicFilters.size(); j++) {
+                LogicalTupleFilter f = logical(FilterOperatorEnum.OR, basicFilters.get(i), basicFilters.get(j));
+                RoaringBitmap r = basicResults.get(i).clone();
+                r.or(basicResults.get(j));
+                assertEquals(r, eval.evaluate(f));
+            }
+        }
+    }
+
+    @Test
+    public void testNotEvaluable() {
+        CompareTupleFilter notEvaluable = compare(colB, FilterOperatorEnum.EQ, 0);
+        assertEquals(null, eval.evaluate(notEvaluable));
+
+        LogicalTupleFilter or = logical(FilterOperatorEnum.OR, basicFilters.get(1), notEvaluable);
+        assertEquals(null, eval.evaluate(or));
+
+        LogicalTupleFilter and = logical(FilterOperatorEnum.AND, basicFilters.get(1), notEvaluable);
+        assertEquals(basicResults.get(1), eval.evaluate(and));
+    }
+
+    public static CompareTupleFilter compare(TblColRef col, TupleFilter.FilterOperatorEnum op, int... ids) {
+        CompareTupleFilter filter = new CompareTupleFilter(op);
+        filter.setNullString(idToStr(Dictionary.NULL_ID[1]));
+        filter.addChild(columnFilter(col));
+        for (int i : ids) {
+            filter.addChild(constFilter(i));
+        }
+        return filter;
+    }
+
+    public static LogicalTupleFilter logical(TupleFilter.FilterOperatorEnum op, TupleFilter... filters) {
+        LogicalTupleFilter filter = new LogicalTupleFilter(op);
+        for (TupleFilter f : filters)
+            filter.addChild(f);
+        return filter;
+    }
+
+    public static ColumnTupleFilter columnFilter(TblColRef col) {
+        return new ColumnTupleFilter(col);
+    }
+
+    public static ConstantTupleFilter constFilter(int id) {
+        return new ConstantTupleFilter(idToStr(id));
+    }
+
+    public static RoaringBitmap set(int... ints) {
+        return RoaringBitmap.bitmapOf(ints);
+    }
+
+    public static String idToStr(int id) {
+        byte[] bytes = new byte[] { (byte) id };
+        return Dictionary.dictIdToString(bytes, 0, bytes.length);
+    }
+
+}


[40/45] incubator-kylin git commit: KYLIN-1034 package roaringbitmap into coprocessor jar

Posted by sh...@apache.org.
KYLIN-1034 package roaringbitmap into coprocessor jar


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

Branch: refs/heads/master
Commit: eca369883563b9c125836a0b23dfc2defd62009e
Parents: 4ab5612
Author: honma <ho...@ebay.com>
Authored: Thu Oct 8 10:41:30 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Thu Oct 8 11:51:00 2015 +0800

----------------------------------------------------------------------
 storage/pom.xml | 13 +++++++++----
 1 file changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/eca36988/storage/pom.xml
----------------------------------------------------------------------
diff --git a/storage/pom.xml b/storage/pom.xml
index 252fd9d..5b2ce8f 100644
--- a/storage/pom.xml
+++ b/storage/pom.xml
@@ -17,7 +17,8 @@
  limitations 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>
 
     <artifactId>kylin-storage</artifactId>
@@ -127,7 +128,8 @@
                             <shadedArtifactAttached>true</shadedArtifactAttached>
                             <shadedClassifierName>coprocessor</shadedClassifierName>
                             <transformers>
-                                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
                             </transformers>
                             <artifactSet>
                                 <includes>
@@ -138,7 +140,8 @@
                                     <include>org.apache.kylin:kylin-storage</include>
                                     <include>org.apache.kylin:kylin-invertedindex</include>
                                     <include>com.ning:compress-lzf</include>
-                                    <include>com.n3twork.druid:extendedset</include>
+                                    <!--<include>com.n3twork.druid:extendedset</include>-->
+                                    <include>org.roaringbitmap:RoaringBitmap</include>
                                     <include>net.sf.trove4j:*</include>
                                     <!-- include>com.fasterxml.jackson.core:*</include -->
                                     <!-- include>org.apache.commons:commons-lang3</include -->
@@ -181,7 +184,9 @@
                             <numUnapprovedLicenses>0</numUnapprovedLicenses>
                             <excludes>
                                 <!-- protobuf -->
-                                <exclude>src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/generated/IIProtos.java</exclude>
+                                <exclude>
+                                    src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/generated/IIProtos.java
+                                </exclude>
                                 <exclude>**/*.proto</exclude>
                             </excludes>
                         </configuration>


[24/45] incubator-kylin git commit: KYLIN-1052 add shaofengshi’s pub key

Posted by sh...@apache.org.
KYLIN-1052 add shaofengshi’s pub key


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

Branch: refs/heads/master
Commit: b22ca0d09643d37e484c71d4190cf1303af1d938
Parents: 80fbd77
Author: shaofengshi <sh...@apache.org>
Authored: Wed Sep 30 15:16:36 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Wed Sep 30 15:33:32 2015 +0800

----------------------------------------------------------------------
 KEYS | 58 ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 58 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b22ca0d0/KEYS
----------------------------------------------------------------------
diff --git a/KEYS b/KEYS
index d5dedb6..06e5b4e 100644
--- a/KEYS
+++ b/KEYS
@@ -71,3 +71,61 @@ EBG4pxGI2XsApbts0AkkINEYAKN+Z/o9BtrpEvodFPn2TBR3D9zPM+XiSZXTZ/gG
 zJ5Mba6u6w6DZa238CtPdRTMh5ZAGhgkWYCrpQJ+CD9qflqNFzpTQ4FtPg8=
 =YeBe
 -----END PGP PUBLIC KEY BLOCK-----
+pub   4096R/6FCF4720 2015-09-29
+uid       [ultimate] Shaofeng Shi <sh...@apache.org>
+sig 3        6FCF4720 2015-09-29  Shaofeng Shi <sh...@apache.org>
+sub   4096R/90EFBBD8 2015-09-29
+sig          6FCF4720 2015-09-29  Shaofeng Shi <sh...@apache.org>
+
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+Comment: GPGTools - https://gpgtools.org
+
+mQINBFYKTLQBEACwI2aNrCpaYAHI4nMjKSVOlBBCVA5mPaLTLeRXPWm9LY1HSjtl
+HAS3GZYijhwhJ9ZiDtHTvZUyZdEAJZf16jNg7p5dhdSfjp/b91a8NfNuoLuSXx7R
+NdBd4xB8IeSjZDGeMw5wqUXrta0w7dNJjEiYs9/4lh3BkA/x/NPRfjkYmHk6wixj
+YGs6rpolACqRBnZ612qoRI7eOrSy9WC6gU3sajHlWiOPHECBF7jNHU6aeQH4YIig
+UufHALU1gdb4o8qH6h9QibvCKoqdI7LwnD4Pn53SXAfpnNpkgH0SLzdpG++argd+
+33IQeo6DCNazkQ//qdRlrXFsriGNEq8Aip+614hyDkS3N20MZzHrX2sfOz19oZN4
+5UNvkz7gbys/L4MJL1RFapjgofXFKr7wNPAdWE+ZvJ4Irw88pM1LmtDSS/m1tyLn
+hz51vHcw/sBo90DisgNjPZUvO9kHXb1LTVVVmL6EdO0PFp9b4qlKGWUk+IGKejI6
+5ooSlMSjvPAYlmQvy+dcl5uVu+n9M/S5gPIWGiPxrJc73SzOZaHedTSVd2g9yJCz
+smuKKni7og19wgPrAEPvmVS/uObp+rmae2R5/EqgBL1PAIxnPCJbqUJp/865ti2K
+atQeua1yntn3cTFCpO6CkSQXk2LjMtLglZl4AOmf+9UmKi/AsefvSD8t+wARAQAB
+tCVTaGFvZmVuZyBTaGkgPHNoYW9mZW5nc2hpQGFwYWNoZS5vcmc+iQI3BBMBCgAh
+BQJWCky0AhsDBQsJCAcDBRUKCQgLBRYCAwEAAh4BAheAAAoJEKvJ+IVvz0cgVE4P
+/RucXCEPWyG3GY7pGtgBuouBfhHhrWKAkVd58818NwLfOkvTWbGuGNWBqQ1CRPGk
+8ZhaNoVx3T8RqIOOyLx+ynOB6iBhPrVOyMsIxb7jh1BnMhe2aGTZGrrQCChGZk2H
+cdxVbCipO4Pw+YdUtJ4Mc3Xm14TkwAygGqrfJEKXgiAWBlvzrKp6twc69jgSd+To
+/UIrSFdPSlpbEpPiMuGKfhrz8OxPYy6N+/AgwIqGub/8NX9cteLCXRQmL5h7Z+6f
+7Wa4XGlEDfxCJFnWKsUYHwP5QHkJPfESGdbYgI+nrN8rv25rtINfWaiBgAsFSg4D
+KYZBXAelumDmLD/Rpn+hhjpeC5f6lPCqap1atHoqLx/s8YI8149on049pzA5t/zK
++G+c8bYf8xm6Ky/dBQP15ExoeeIpjC5A5vNlxenaEXXCya50J4+JLC5DTi9xRU7R
+Il3InFjeRmx1hPeyuYSY2irzBFvDhdt7U0wvwERS0lHMkwvB/MPrEwuB/II2tsFQ
+eZvHFhROYh4H8TqE9qICRtcdqNR8BXsZdEJtno4YH//b58130qJSNwcBF/mt+IcZ
+ryVQe3Y87evcF8RWTvWd4KQxe8BGoGQ5o8U230bxJ2bNYl+6O8Xzg+Q4HGpX4DGw
+EUVf4fE3n9O6kH0p9P6RAgnvlsKtsyubxlWz2zNXFuteuQINBFYKTLQBEADbKpjV
+NxaIuUg+wYSLP5erFvHwqDT5OJCj2y4wbwTzVb/1wZL2QwIPzb8lmlJuRsHo1E+j
+bjQE0I1RWiCAdekIpljBHc7fLsYhV0ONMgQZxlh3lpZCIftKmIPX4Eh93bhDBtRi
+CnYcWa/MDEH8rccwZaz0j+2OstIAv7x5/mUC55eutFtH4m5wRkbAjgcZbu6h0MNf
+KbBB4sOfHyx9L1bDIph42Hoh8rp+sMDYNf5Hnxr3Qt4ojANtQgQJzXudI6GzDZHy
+laIle5nJudyPy/TdF/8QXsUT62fbO8W8e4Q8Ljx192RULGoVYlz8WPkXF1vbL8Yb
+mMJyQ9YCgc5jIg/NusMFT0OXHV3YNQ/tZNh57i27pGO4RolRRbrXPrE/npIGBrsR
+8myPu3ussAMSWF2ejhWupKYS943ujNDOhKqC88R5mwHSzDpfytMUMsElwzrioAPV
+EiCCcJ8/Q0u8RAlF9A629iquIA5pp4Ga+9CLHdeGUohmnpIePz71+sXLW+VUOUlM
++euttiH/ltnRV0LKvS0w6sSnfOFOcOGZKxJ1FJhD5rx86dy7vjrWveOfoCNKjm3a
+jeTOfptvuXzJy0rEgVsKBRLCosjJIVgyWR3UkyXFCq3HfS0IutmdKMzxO+/J6t0c
+pnUsIHEiMlV/2TjwM/icFfUSDzAM/D+OoYe/IQARAQABiQIfBBgBCgAJBQJWCky0
+AhsMAAoJEKvJ+IVvz0cgJ0QQAJt5YtBkMKzPc/QGQo9T86ArCijnt2Z0P6d3S0Qr
+dye87F8ZmM3MRl/Bd7EpZ/+l3157FTX3jBKBrWsKTFcrG5IXrbMTS81pi555AwBm
+HcwCDBdFaLBKlqmQemgNKgnOGt+/G3w1oetPlbkKQEgFjCFfMF8CS8HdNqJU8pmd
+RE3IHMEvwrqx1dbGntRvUrSxs+rRgjJHympIAwIPdLHO6YSIrsbZEghUeVzd/5xi
+cEN06IieGY5t7PImpxeTmYAAdbQsN2GvYOG8NdOY7GrQ2Pu83mreb5Ma/Ui5UtPd
+3KWanXTP4lliY1DYkphp8USjdTul+srOK6X4dQ4y3EYyOzOytvMNV+G4bcYoTnoS
+gUkT1p0oHArxrQej3Py+no4Y5FuYEPJdZ3oHOhkqgn9j/UZBRxfc6VMRAD3ymRXC
+I7eDwdwOryyLBvA16L1+gs2kc5cbbfeFgfFiEFlMfT3eUN9mu5cQ3dNCQ119fcJp
+cu3E67ykSeibobuwNFj+JCilCUGrV5KN3FPVet3TX8F09O9z0CJwjmiK6+JkvVV8
+GB8WCCu0wEwSHYlTZkzOmXMNYZjk8fNYkWplWbF6xNPs1A00XqDwsa/V0F9EzJV+
+nYr5b8EWaM9qsjRh2aFPFtV+E/cSc7sdJxIY/TSdMfqhB13l7LTU/NwmgjE2TTUH
+WYIE
+=6yhP
+-----END PGP PUBLIC KEY BLOCK-----


[37/45] incubator-kylin git commit: add powered by page

Posted by sh...@apache.org.
add powered by page


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

Branch: refs/heads/master
Commit: 875ca218eaac34e0826b2a21de6b6265a3969bf5
Parents: 5b98b7e
Author: lukehan <lu...@apache.org>
Authored: Thu Oct 8 00:25:35 2015 +0800
Committer: lukehan <lu...@apache.org>
Committed: Thu Oct 8 00:25:35 2015 +0800

----------------------------------------------------------------------
 website/_docs/gettingstarted/terminology.md | 5 ++---
 website/community/poweredby.md              | 2 +-
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/875ca218/website/_docs/gettingstarted/terminology.md
----------------------------------------------------------------------
diff --git a/website/_docs/gettingstarted/terminology.md b/website/_docs/gettingstarted/terminology.md
index ef38df8..ac4a19d 100644
--- a/website/_docs/gettingstarted/terminology.md
+++ b/website/_docs/gettingstarted/terminology.md
@@ -3,11 +3,10 @@ layout: docs
 title:  "Terminology"
 categories: gettingstarted
 permalink: /docs/gettingstarted/terminology.html
-version: v0.7.2
+version: v1.0
 since: v0.5.x
 ---
-
-  
+ 
 
 Here are some terms we are using in Apache Kylin, please check them for your reference.   
 They are basic knowledge of Apache Kylin which also will help to well understand such concerpt, term, knowledge, theory and others about Data Warehouse, Business Intelligence for analycits. 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/875ca218/website/community/poweredby.md
----------------------------------------------------------------------
diff --git a/website/community/poweredby.md b/website/community/poweredby.md
index e600cec..a76e427 100644
--- a/website/community/poweredby.md
+++ b/website/community/poweredby.md
@@ -11,7 +11,7 @@ _Send a quick description of your organization and usage to the [mailing list](m
 
 __Companies__
 
-* [eBay](http://www.ebay.com), Kylin was developed inside eBay for Big Data Analytics on Hadoop and then contributed to open source world in Oct 2014, accepted as Apache Incubator project in Nov 2014. Apache Kylin is used at eBay for OLAP on Hadoop. This powers various data products including Behavior Analytics, Traffic Reporting, Account Manager Application and Streaming Dashboard.
+* [eBay](http://www.ebay.com), Apache Kylin is used at eBay for Big Data Analytics on Hadoop. This powers various data products including Behavior Analytics, Traffic Reporting, Account Manager Application and Streaming Dashboard.
 * [MiningLAMP](http://www.mininglamp.com/), Kylin is part of MiningLAMP's Big Data Analytics product offering OLAP capability 
 * [Meituan](http://www.meituan.com)
 * 


[27/45] incubator-kylin git commit: KYLIN-457 Fix double log lines in kylin.log

Posted by sh...@apache.org.
KYLIN-457 Fix double log lines in kylin.log


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

Branch: refs/heads/master
Commit: 24e59fc7e54e354a50c2e263d039a849323b168a
Parents: f6f67e7
Author: Yang Li <li...@apache.org>
Authored: Sun Oct 4 20:51:36 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sun Oct 4 20:51:36 2015 +0800

----------------------------------------------------------------------
 bin/kylin.sh                                    |  3 +-
 .../src/main/resources/kylin-log4j.properties   |  1 +
 .../src/main/resources/kylin-log4j.properties   | 55 --------------------
 .../resources/kylin-server-log4j.properties     | 51 ++++++++++++++++++
 server/src/main/webapp/WEB-INF/web.xml          |  6 +--
 5 files changed, 56 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24e59fc7/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/bin/kylin.sh b/bin/kylin.sh
index c11ddf9..1334ed1 100644
--- a/bin/kylin.sh
+++ b/bin/kylin.sh
@@ -63,7 +63,6 @@ then
     hbase ${KYLIN_EXTRA_START_OPTS} \
     -Djava.util.logging.config.file=${tomcat_root}/conf/logging.properties \
     -Djava.util.logging.manager=org.apache.juli.ClassLoaderLogManager \
-    -Dlog4j.configuration=kylin-log4j.properties \
     -Dorg.apache.tomcat.util.buf.UDecoder.ALLOW_ENCODED_SLASH=true \
     -Dorg.apache.catalina.connector.CoyoteAdapter.ALLOW_BACKSLASH=true \
     -Djava.endorsed.dirs=${tomcat_root}/endorsed  \
@@ -116,7 +115,7 @@ then
 
     export HBASE_CLASSPATH=${KYLIN_HOME}/lib/*:$hive_dependency:${HBASE_CLASSPATH}
 
-    exec hbase -Dlog4j.configuration=kylin-log4j.properties "$@"
+    exec hbase "$@"
 
 else
     echo "usage: kylin.sh start or kylin.sh stop"

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24e59fc7/common/src/main/resources/kylin-log4j.properties
----------------------------------------------------------------------
diff --git a/common/src/main/resources/kylin-log4j.properties b/common/src/main/resources/kylin-log4j.properties
index d499713..3772972 100644
--- a/common/src/main/resources/kylin-log4j.properties
+++ b/common/src/main/resources/kylin-log4j.properties
@@ -25,3 +25,4 @@ log4j.appender.stdout.layout.ConversionPattern=L4J [%d{yyyy-MM-dd HH:mm:ss,SSS}]
 
 #log4j.logger.org.apache.hadoop=ERROR
 log4j.logger.org.apache.kylin=DEBUG
+log4j.logger.org.springframework=WARN

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24e59fc7/server/src/main/resources/kylin-log4j.properties
----------------------------------------------------------------------
diff --git a/server/src/main/resources/kylin-log4j.properties b/server/src/main/resources/kylin-log4j.properties
deleted file mode 100644
index d7dc3bf..0000000
--- a/server/src/main/resources/kylin-log4j.properties
+++ /dev/null
@@ -1,55 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-#define appenders
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
-
-log4j.appender.file=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.file.layout=org.apache.log4j.PatternLayout
-log4j.appender.file.File=${catalina.home}/logs/kylin.log
-log4j.appender.file.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
-log4j.appender.file.Append=true
-
-log4j.appender.query=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.query.layout=org.apache.log4j.PatternLayout
-log4j.appender.query.File=${catalina.home}/logs/kylin_query.log
-log4j.appender.query.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
-log4j.appender.query.Append=true
-
-log4j.appender.job=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.job.layout=org.apache.log4j.PatternLayout
-log4j.appender.job.File=${catalina.home}/logs/kylin_job.log
-log4j.appender.job.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
-log4j.appender.job.Append=true
-
-#overall config
-log4j.rootLogger=INFO,stdout,file
-log4j.logger.org.apache.kylin=DEBUG
-log4j.logger.org.springframework=WARN
-
-#query config
-log4j.logger.org.apache.kylin.rest.controller.QueryController=DEBUG, query
-log4j.logger.org.apache.kylin.rest.service.QueryService=DEBUG, query
-log4j.logger.org.apache.kylin.query=DEBUG, query
-log4j.logger.org.apache.kylin.storage=DEBUG, query
-
-#job config
-log4j.logger.org.apache.kylin.rest.controller.JobController=DEBUG, job
-log4j.logger.org.apache.kylin.rest.service.JobService=DEBUG, job
-log4j.logger.org.apache.kylin.job=DEBUG, job

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24e59fc7/server/src/main/resources/kylin-server-log4j.properties
----------------------------------------------------------------------
diff --git a/server/src/main/resources/kylin-server-log4j.properties b/server/src/main/resources/kylin-server-log4j.properties
new file mode 100644
index 0000000..6ccd161
--- /dev/null
+++ b/server/src/main/resources/kylin-server-log4j.properties
@@ -0,0 +1,51 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+#define appenders
+log4j.appender.file=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.File=${catalina.home}/logs/kylin.log
+log4j.appender.file.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
+log4j.appender.file.Append=true
+
+log4j.appender.query=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.query.layout=org.apache.log4j.PatternLayout
+log4j.appender.query.File=${catalina.home}/logs/kylin_query.log
+log4j.appender.query.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
+log4j.appender.query.Append=true
+
+log4j.appender.job=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.job.layout=org.apache.log4j.PatternLayout
+log4j.appender.job.File=${catalina.home}/logs/kylin_job.log
+log4j.appender.job.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
+log4j.appender.job.Append=true
+
+#overall config
+log4j.rootLogger=INFO,file
+log4j.logger.org.apache.kylin=DEBUG
+log4j.logger.org.springframework=WARN
+
+#query config
+log4j.logger.org.apache.kylin.rest.controller.QueryController=DEBUG, query
+log4j.logger.org.apache.kylin.rest.service.QueryService=DEBUG, query
+log4j.logger.org.apache.kylin.query=DEBUG, query
+log4j.logger.org.apache.kylin.storage=DEBUG, query
+
+#job config
+log4j.logger.org.apache.kylin.rest.controller.JobController=DEBUG, job
+log4j.logger.org.apache.kylin.rest.service.JobService=DEBUG, job
+log4j.logger.org.apache.kylin.job=DEBUG, job

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/24e59fc7/server/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/server/src/main/webapp/WEB-INF/web.xml b/server/src/main/webapp/WEB-INF/web.xml
index ca5f6d9..d0b413a 100644
--- a/server/src/main/webapp/WEB-INF/web.xml
+++ b/server/src/main/webapp/WEB-INF/web.xml
@@ -19,7 +19,7 @@ limitations under the License. See accompanying LICENSE file.
 		  http://java.sun.com/xml/ns/javaee/web-app_2_5.xsd"
          version="2.5">
 
-    <display-name>Kylin RESt Service</display-name>
+    <display-name>Kylin REST Service</display-name>
 
     <welcome-file-list>
         <welcome-file>/index.html</welcome-file>
@@ -32,7 +32,7 @@ limitations under the License. See accompanying LICENSE file.
 
     <context-param>
         <param-name>log4jConfigLocation</param-name>
-        <param-value>classpath:kylin-log4j.properties</param-value>
+        <param-value>classpath:kylin-server-log4j.properties</param-value>
     </context-param>
     <context-param>
         <param-name>contextConfigLocation</param-name>
@@ -44,7 +44,7 @@ limitations under the License. See accompanying LICENSE file.
     </context-param>
 
     <listener>
-        <listener-class>org.springframework.web.util.Log4jConfigListener</listener-class>
+        <listener-class>org.apache.kylin.rest.util.Log4jConfigListener</listener-class>
     </listener>
 
     <listener>


[44/45] incubator-kylin git commit: [maven-release-plugin] prepare release kylin-1.1-incubating

Posted by sh...@apache.org.
[maven-release-plugin] prepare release kylin-1.1-incubating


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

Branch: refs/heads/master
Commit: bac74fb45f41c09e4346c08cf2ef9df7c158c58c
Parents: 1955a2f
Author: shaofengshi <sh...@apache.org>
Authored: Fri Oct 9 18:07:58 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Oct 9 18:07:58 2015 +0800

----------------------------------------------------------------------
 atopcalcite/pom.xml   | 2 +-
 common/pom.xml        | 2 +-
 cube/pom.xml          | 2 +-
 dictionary/pom.xml    | 2 +-
 invertedindex/pom.xml | 2 +-
 jdbc/pom.xml          | 2 +-
 job/pom.xml           | 2 +-
 metadata/pom.xml      | 2 +-
 monitor/pom.xml       | 2 +-
 pom.xml               | 4 ++--
 query/pom.xml         | 2 +-
 server/pom.xml        | 2 +-
 storage/pom.xml       | 8 +++-----
 13 files changed, 16 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/atopcalcite/pom.xml
----------------------------------------------------------------------
diff --git a/atopcalcite/pom.xml b/atopcalcite/pom.xml
index 88ac41d..0f99a3d 100644
--- a/atopcalcite/pom.xml
+++ b/atopcalcite/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.1-incubating-SNAPSHOT</version>
+        <version>1.1-incubating</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index d61531a..23543b8 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.1-incubating-SNAPSHOT</version>
+        <version>1.1-incubating</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/cube/pom.xml
----------------------------------------------------------------------
diff --git a/cube/pom.xml b/cube/pom.xml
index e5c1f91..97b9fa8 100644
--- a/cube/pom.xml
+++ b/cube/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.1-incubating-SNAPSHOT</version>
+        <version>1.1-incubating</version>
     </parent>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/dictionary/pom.xml
----------------------------------------------------------------------
diff --git a/dictionary/pom.xml b/dictionary/pom.xml
index 30c3468..763b918 100644
--- a/dictionary/pom.xml
+++ b/dictionary/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.1-incubating-SNAPSHOT</version>
+        <version>1.1-incubating</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/invertedindex/pom.xml
----------------------------------------------------------------------
diff --git a/invertedindex/pom.xml b/invertedindex/pom.xml
index 0eaaf1e..8ff69f0 100644
--- a/invertedindex/pom.xml
+++ b/invertedindex/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>kylin</artifactId>
         <groupId>org.apache.kylin</groupId>
-        <version>1.1-incubating-SNAPSHOT</version>
+        <version>1.1-incubating</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>kylin-invertedindex</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index 0db4419..7ca71db 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -27,7 +27,7 @@
 	<parent>
 		<groupId>org.apache.kylin</groupId>
 		<artifactId>kylin</artifactId>
-		<version>1.1-incubating-SNAPSHOT</version>
+		<version>1.1-incubating</version>
 	</parent>
 
 	<properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/job/pom.xml
----------------------------------------------------------------------
diff --git a/job/pom.xml b/job/pom.xml
index ba73760..ae2e083 100644
--- a/job/pom.xml
+++ b/job/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.1-incubating-SNAPSHOT</version>
+        <version>1.1-incubating</version>
     </parent>
 
     <artifactId>kylin-job</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/metadata/pom.xml
----------------------------------------------------------------------
diff --git a/metadata/pom.xml b/metadata/pom.xml
index 7970fee..477efb8 100644
--- a/metadata/pom.xml
+++ b/metadata/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.1-incubating-SNAPSHOT</version>
+        <version>1.1-incubating</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/monitor/pom.xml
----------------------------------------------------------------------
diff --git a/monitor/pom.xml b/monitor/pom.xml
index 566c873..e359fa0 100644
--- a/monitor/pom.xml
+++ b/monitor/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.1-incubating-SNAPSHOT</version>
+        <version>1.1-incubating</version>
     </parent>
 
     <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index aca1ff1..a54d34c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
     <groupId>org.apache.kylin</groupId>
     <artifactId>kylin</artifactId>
     <packaging>pom</packaging>
-    <version>1.1-incubating-SNAPSHOT</version>
+    <version>1.1-incubating</version>
     <name>Kylin:HadoopOLAPEngine</name>
     <url>http://kylin.incubator.apache.org</url>
     <inceptionYear>2014</inceptionYear>
@@ -141,7 +141,7 @@
         <connection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-kylin.git</connection>
         <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-kylin.git</developerConnection>
         <url>https://git-wip-us.apache.org/repos/asf/incubator-kylin.git</url>
-        <tag>apache-kylin-0.7.1-incubating</tag>
+        <tag>kylin-1.1-incubating</tag>
     </scm>
 
     <dependencyManagement>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/query/pom.xml
----------------------------------------------------------------------
diff --git a/query/pom.xml b/query/pom.xml
index 3c3601a..f76b6fb 100644
--- a/query/pom.xml
+++ b/query/pom.xml
@@ -28,7 +28,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.1-incubating-SNAPSHOT</version>
+        <version>1.1-incubating</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index eed7068..b8b3333 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -28,7 +28,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.1-incubating-SNAPSHOT</version>
+        <version>1.1-incubating</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bac74fb4/storage/pom.xml
----------------------------------------------------------------------
diff --git a/storage/pom.xml b/storage/pom.xml
index 5b2ce8f..19dc5de 100644
--- a/storage/pom.xml
+++ b/storage/pom.xml
@@ -17,8 +17,7 @@
  limitations 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>
 
     <artifactId>kylin-storage</artifactId>
@@ -27,7 +26,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>1.1-incubating-SNAPSHOT</version>
+        <version>1.1-incubating</version>
     </parent>
 
     <dependencies>
@@ -128,8 +127,7 @@
                             <shadedArtifactAttached>true</shadedArtifactAttached>
                             <shadedClassifierName>coprocessor</shadedClassifierName>
                             <transformers>
-                                <transformer
-                                        implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
                             </transformers>
                             <artifactSet>
                                 <includes>


[15/45] incubator-kylin git commit: KYLIN-1048 fix CPU and memory killer in Cuboid.findById()

Posted by sh...@apache.org.
KYLIN-1048 fix CPU and memory killer in Cuboid.findById()

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

Branch: refs/heads/master
Commit: e82eb002f1b228999d5a18ebd928625773bb2b29
Parents: e924a2d
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 28 15:30:52 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 28 15:30:52 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/cube/cuboid/Cuboid.java    | 113 ++++++++++++++++---
 1 file changed, 100 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e82eb002/cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java b/cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
index 93fa810..9080115 100644
--- a/cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
+++ b/cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
@@ -18,23 +18,15 @@
 
 package org.apache.kylin.cube.cuboid;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.RowKeyColDesc;
-import org.apache.kylin.cube.model.RowKeyDesc;
+import org.apache.kylin.cube.model.*;
 import org.apache.kylin.cube.model.RowKeyDesc.AggrGroupMask;
 import org.apache.kylin.cube.model.RowKeyDesc.HierarchyMask;
 import org.apache.kylin.metadata.model.TblColRef;
 
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+
 /**
  * @author George Song (ysong1)
  */
@@ -55,6 +47,10 @@ public class Cuboid implements Comparable<Cuboid> {
         Cuboid cuboid = cubeCache.get(cuboidID);
         if (cuboid == null) {
             long validCuboidID = translateToValidCuboid(cube, cuboidID);
+            if (Cuboid.isValid(cube, validCuboidID) == false) {
+                throw new RuntimeException("Didn't find a valid cuboid: " + validCuboidID);
+            }
+
             cuboid = new Cuboid(cube, cuboidID, validCuboidID);
             cubeCache.put(cuboidID, cuboid);
         }
@@ -92,8 +88,99 @@ public class Cuboid implements Comparable<Cuboid> {
         return cube.getRowkey().getFullMask();
     }
 
+    private static long translateToValidCuboid(CubeDesc cubeDesc, long cuboidID) {
+        // add mandantory
+        RowKeyDesc rowkey = cubeDesc.getRowkey();
+        long mandatoryColumnMask = rowkey.getMandatoryColumnMask();
+        if (cuboidID < mandatoryColumnMask) {
+            cuboidID = cuboidID | mandatoryColumnMask;
+        }
+
+        // add hierarchy
+        for (DimensionDesc dimension : cubeDesc.getDimensions()) {
+            HierarchyDesc[] hierarchies = dimension.getHierarchy();
+            boolean found = false;
+            long result = 0;
+            if (hierarchies != null && hierarchies.length > 0) {
+                for (int i = hierarchies.length - 1; i >= 0; i--) {
+                    TblColRef hColumn = hierarchies[i].getColumnRef();
+                    Integer index = rowkey.getColumnBitIndex(hColumn);
+                    long bit = 1L << index;
+
+                    if ((rowkey.getTailMask() & bit) > 0)
+                        continue; // ignore levels in tail, they don't participate
+
+                    if ((bit & cuboidID) > 0) {
+                        found = true;
+                    }
+
+                    if (found == true) {
+                        result = result | bit;
+                    }
+                }
+                cuboidID = cuboidID | result;
+            }
+        }
+
+        // find the left-most aggregation group
+        long cuboidWithoutMandatory = cuboidID & ~rowkey.getMandatoryColumnMask();
+        long leftover;
+        for (AggrGroupMask mask : rowkey.getAggrGroupMasks()) {
+            if ((cuboidWithoutMandatory & mask.uniqueMask) == mask.uniqueMask) {
+                leftover = cuboidWithoutMandatory & ~mask.groupMask;
+
+                if (leftover == 0) {
+                    return cuboidID;
+                }
+
+                if (leftover != 0) {
+                    cuboidID = cuboidID | mask.leftoverMask;
+                    return cuboidID;
+                }
+            }
+        }
+
+        // doesn't have column in aggregation groups
+        leftover = cuboidWithoutMandatory & rowkey.getTailMask();
+        if (leftover == 0) {
+            // doesn't have column in tail group
+            if (cuboidWithoutMandatory != 0) {
+                return cuboidID;
+            } else {
+                // no column (except mandatory), add one column
+                long toAddCol = (1 << (BitSet.valueOf(new long[] { rowkey.getTailMask() }).cardinality()));
+                // check if the toAddCol belongs to any hierarchy
+                List<HierarchyMask> hierarchyMaskList = rowkey.getHierarchyMasks();
+                if (hierarchyMaskList != null && hierarchyMaskList.size() > 0) {
+                    for (HierarchyMask hierarchyMasks : hierarchyMaskList) {
+                        long result = toAddCol & hierarchyMasks.fullMask;
+                        if (result > 0) {
+                            // replace it with the root col in hierarchy
+                            toAddCol = hierarchyMasks.allMasks[0];
+                            break;
+                        }
+                    }
+                }
+                cuboidID = cuboidID | toAddCol;
+                return cuboidID;
+            }
+        }
+
+        // has column in tail group
+        cuboidID = cuboidID | rowkey.getTailMask();
+        return cuboidID;
+
+    }
+
     // Breadth-First-Search
-    private static long translateToValidCuboid(CubeDesc cube, long cuboidID) {
+
+    /**
+     * @deprecated due to poor performance
+     * @param cube
+     * @param cuboidID
+     * @return
+     */
+    private static long translateToValidCuboidDeprecated(CubeDesc cube, long cuboidID) {
         if (Cuboid.isValid(cube, cuboidID)) {
             return cuboidID;
         }


[39/45] incubator-kylin git commit: KYLIN-1061 "kylin.sh start" should check whether kylin has already been running

Posted by sh...@apache.org.
KYLIN-1061 "kylin.sh start" should check whether kylin has already been running

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

Branch: refs/heads/master
Commit: 4ab5612b0bb66b1640f5854a3ef825213d41a52c
Parents: 1696f70
Author: shaofengshi <sh...@apache.org>
Authored: Thu Oct 8 10:43:51 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Oct 8 10:43:51 2015 +0800

----------------------------------------------------------------------
 bin/kylin.sh | 40 ++++++++++++++++++++++++++--------------
 1 file changed, 26 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4ab5612b/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/bin/kylin.sh b/bin/kylin.sh
index 1334ed1..021e494 100644
--- a/bin/kylin.sh
+++ b/bin/kylin.sh
@@ -25,6 +25,16 @@ mkdir -p ${KYLIN_HOME}/logs
 if [ $1 == "start" ]
 then
 
+    if [ -f "${KYLIN_HOME}/pid" ]
+    then
+        PID=`cat $KYLIN_HOME/pid`
+        if ps -p $PID > /dev/null
+        then
+          echo "Kylin is running, stop it first"
+          exit 1
+        fi
+    fi
+    
     tomcat_root=${dir}/../tomcat
     export tomcat_root
 
@@ -85,23 +95,25 @@ then
 # stop command
 elif [ $1 == "stop" ]
 then
-    if [ ! -f "${KYLIN_HOME}/pid" ]
-    then
-        echo "kylin is not running, please check"
-        exit 1
-    fi
-    pid=`cat ${KYLIN_HOME}/pid`
-    if [ "$pid" = "" ]
+    if [ -f "${KYLIN_HOME}/pid" ]
     then
-        echo "kylin is not running, please check"
-        exit 1
+        PID=`cat $KYLIN_HOME/pid`
+        if ps -p $PID > /dev/null
+        then
+           echo "stopping Kylin:$PID"
+           kill $PID
+           rm ${KYLIN_HOME}/pid
+           exit 0
+        else
+           echo "Kylin is not running, please check"
+           exit 1
+        fi
+        
     else
-        echo "stopping kylin:$pid"
-        kill $pid
+        echo "Kylin is not running, please check"
+        exit 1    
     fi
-    rm ${KYLIN_HOME}/pid
-    exit 0
-
+    
 # tool command
 elif [[ $1 = org.apache.kylin.* ]]
 then


[14/45] incubator-kylin git commit: Small update on blog for hybrid model

Posted by sh...@apache.org.
Small update on blog for hybrid model


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

Branch: refs/heads/master
Commit: e924a2db0e9025d9e370403579e4f789ec4412f4
Parents: 38f4fd0
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 26 22:22:08 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 28 09:09:03 2015 +0800

----------------------------------------------------------------------
 website/_posts/blog/2015-09-22-hybrid-model.md | 56 ++++++++++++---------
 1 file changed, 32 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e924a2db/website/_posts/blog/2015-09-22-hybrid-model.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-09-22-hybrid-model.md b/website/_posts/blog/2015-09-22-hybrid-model.md
index 1e0f4c7..d966fa5 100644
--- a/website/_posts/blog/2015-09-22-hybrid-model.md
+++ b/website/_posts/blog/2015-09-22-hybrid-model.md
@@ -6,21 +6,22 @@ author: Shaofeng Shi
 categories: blog
 ---
 
-**Apache Kylin v1.0 introduces a new realization "hybrid model" (also called "dynamic model"); This post introduces the concept and how to define it.**
+**Apache Kylin v1.0 introduces a new realization "hybrid model" (also called "dynamic model"); This post introduces the concept and how to create a hybrid instance.**
 
 # Problem
 
-For incoming SQL queries, Kylin picks ONE (and only ONE) realization to serve the query; Before the "hybrid", there is only one type of realization open for user: Cube. That to say, only 1 Cube would be selected to answer a query;
+For incoming SQL queries, Kylin picks one (and only one) realization to serve the query; Before the "hybrid", there is only one type of realization open for user: Cube. That to say, only 1 Cube would be selected to answer a query;
 
 Now let's start with a sample case; Assume user has a Cube called "Cube_V1", it has been built for a couple of months; Now the user wants to add new dimension or metrics to fulfill their business need; So he created a new Cube named "Cube_V2"; 
 
-Due to some reason user wants to keep the data of "Cube_V1", and expects to build "Cube_V2" from the end date of "Cube_V1"; The possible reasons include:
+Due to some reason user wants to keep "Cube_V1", and expects to build "Cube_V2" from the end date of "Cube_V1"; Possible reasons include:
 
 * History source data has been dropped from Hadoop, not possible to build "Cube_V2" from the very beginning;
 * The cube is large, rebuilding takes very long time;
-* New dimension/metrics is only feasible for the new date, or user feels fine if they were absent for old cube; etc.
+* New dimension/metrics is only available or applied since some day;
+* User feels okay that the result is empty for old days when the query uses new dimensions/metrics.
 
-For some queries that don't use the new measure and metrics, user hopes both "Cube_V1" and "Cube_V2" can be scanned to get a full result, such as "select count(*)...", "select sum(price)..."; With such a background, the "hybrid model" is introduced in Kylin;
+For the queries against the common dimensions/metrics, user expects both "Cube_V1" and "Cube_V2" be scanned to get a full result set; Under such a background, the "hybrid model" is introduced to solve this problem.
 
 ## Hybrid Model
 
@@ -28,11 +29,11 @@ Hybrid model is a new realization which is a composite of one or multiple other
 
 ![]( /images/blog/hybrid-model.png)
 
-Hybrid doesn't have its real storage; It is just like a virtual database view over tables; It acts as a delegator who delegates the requests to its children realizations.
+Hybrid doesn't have its real storage; It is like a virtual database view over the tables; A hybrid instance acts as a delegator who forward the requests to its children realizations and then union the results when gets back from them.
 
-## How to add a Hybrid model
+## How to add a hybrid instance
 
-As there is no UI for creating/editing hybrid model, if have the need, you need manually edit Kylin metadata;
+So far there is no UI for creating/editing hybrid; if have the need, you need manually edit Kylin metadata;
 
 ### Step 1: Take a backup of kylin metadata store 
 
@@ -45,20 +46,20 @@ $KYLIN_HOME/bin/metastore.sh backup
 
 A backup folder will be created, assume it is $KYLIN_HOME/metadata_backup/2015-09-25/
  
-### Step 2: Create sub-folder "hybrid" in the metadata folder,
+### Step 2: Create sub-folder "hybrid"
 
 ```
 mkdir -p $KYLIN_HOME/metadata_backup/2015-09-25/hybrid
 ```
 
-### Step 3: Create a hybrid json file: 
+### Step 3: Create a hybrid instance json file: 
 
 ```
 vi $KYLIN_HOME/metadata_backup/2015-09-25/hybrid/my_hybrid.json
 
 ```
 
-Input content like this:
+Input content like below, the "name" and "uuid" need be unique:
 
 ```
 {
@@ -80,7 +81,7 @@ Input content like this:
 Here "Cube_V1" and "Cube_V2" are the cubes that you want to combine.
 
 
-### Step 4: Add hybrid model to project
+### Step 4: Add hybrid instance to project
 
 Open project json file (for example project "default") with text editor:
 
@@ -89,7 +90,7 @@ vi $KYLIN_HOME/metadata_backup/2015-09-25/project/default.json
 
 ```
 
-In the "realizations" array, add one entry like:
+In the "realizations" array, add one entry like below, the type need be "HYBRID", "realization" is the name of the hybrid instance:
 
 ```
     {
@@ -105,24 +106,31 @@ In the "realizations" array, add one entry like:
   $KYLIN_HOME/bin/metastore.sh restore $KYLIN_HOME/metadata_backup/2015-09-25/
 
 ```
+Please note, the "restore" action will upload the metadata from local to remote hbase store, which may overwrite the changes in remote; So please do this when there is no metadata change from Kylin server during this period (no building job, no cube creation/update, etc), or only pickup the changed files to an empty local folder before run "restore";
 
 ### Step 6: Reload metadata
 
-Restart Kylin server, or click "Reload metadata" in the "Admin" tab on Kylin web UI to load the changes; Ideally the hybrid will start to work; You can do some verifications.
+Restart Kylin server, or click "Reload metadata" in the "Admin" tab on Kylin web UI to load the changes; Ideally the hybrid will start to work; You can do some verifications by writing some SQLs.
 
 ## FAQ:
 
-**Question 1**: when will hybrid be selected to serve query?
-If one of the cube can answer the query, the hybrid which has it as a child will be selected;
+**Question 1**: When will hybrid be selected to answer a SQL query?
+If one of its underlying cube can answer the query, the hybrid will be selected; 
 
-**Question 2**: how hybrid to answer the query?
-Hybrid will delegate the query to each of its child realization (if it is capable); And then return all the results to query engine; Query engine will aggregate before return to user;
+**Question 2**: How hybrid to answer the query?
+Hybrid will delegate the query to each of its children realizations; If a child cube is capable for this query (match all dimensions/metrics), it will return the results to the hybrid, otherwise it will be skipped; Finally query engine will aggregate the data from hybrid before return to user;
 
-**Question 3**: will hybrid check the data duplication?
-No; it depends on you to ensure the cubes in a hybrid don't have date/time range duplication; For example, the "Cube_V1" is ended at 2015-9-20 (including), the "Cube_V2" should start from 2015-9-21 or later;
+**Question 3**: Will hybrid check the date/time duplication?
+No; it depends on user to ensure the cubes in a hybrid don't have date/time range duplication; For example, the "Cube_V1" is ended at 2015-9-20 (excluding), the "Cube_V2" should start from 2015-9-20 (including); 
 
-**Question 4**: will hybrid restrict the children cubes having the same data model?
-No; hybrid doesn't check the cube's fact/lookup tables and join conditions at all;
+**Question 4**: Will hybrid restrict the children cubes having the same data model?
+No; To provide as much as flexibility, hybrid doesn't check whether the children cubes' fact/lookup tables and join conditions are the same; But user should understand what they're doing to avoid unexpected behavior.
 
-**Question 5**: can hybrid have another hybrid as child?
-No; didn't see the need; so far it assumes all children are Cubes;
+**Question 5**: Can hybrid have another hybrid as child?
+No; we don't see the need; so far it assumes all children are Cubes;
+
+**Question 6**: Can I use hybrid to join multiple cubes?
+No; the purpose of hybrid is to consolidate history cube and new cube, something like a "union", not "join";
+
+**Question 7**: If a child cube is disabled, will it be scanned via the hybrid?
+No; hybrid instance will check the child realization's status before sending query to it; so if the cube is disabled, it will not be scanned. 
\ No newline at end of file


[12/45] incubator-kylin git commit: KYLIN-967 kill long running query when low memory detected

Posted by sh...@apache.org.
KYLIN-967 kill long running query when low memory detected


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

Branch: refs/heads/master
Commit: c611d7eefe78e8b84705b6b0e3d4af29e1686b9e
Parents: bad57cb
Author: shaofengshi <sh...@apache.org>
Authored: Fri Sep 25 16:54:43 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Sep 25 23:31:31 2015 +0800

----------------------------------------------------------------------
 .../kylin/rest/service/BadQueryDetector.java    | 37 ++++++++++++++------
 .../rest/service/BadQueryDetectorTest.java      |  6 ++--
 2 files changed, 29 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c611d7ee/server/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java b/server/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java
index 5f5b247..d3315c0 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java
@@ -20,6 +20,7 @@ package org.apache.kylin.rest.service;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.kylin.rest.request.SQLRequest;
@@ -36,19 +37,21 @@ public class BadQueryDetector extends Thread {
     private final long detectionInterval;
     private final int alertMB;
     private final int alertRunningSec;
+    private final int killRunningSec;
 
     private ArrayList<Notifier> notifiers = new ArrayList<Notifier>();
 
     public BadQueryDetector() {
-        this(60 * 1000, 100, 60); // 1 minute, 100 MB, 60 seconds
+        this(60 * 1000, 100, 60, 5 * 60); // 1 minute, 100 MB, 60 seconds, 5 minutes
     }
 
-    public BadQueryDetector(long detectionInterval, int alertMB, int alertRunningSec) {
+    public BadQueryDetector(long detectionInterval, int alertMB, int alertRunningSec, int killRunningSec) {
         super("BadQueryDetector");
         this.setDaemon(true);
         this.detectionInterval = detectionInterval;
         this.alertMB = alertMB;
         this.alertRunningSec = alertRunningSec;
+        this.killRunningSec = killRunningSec;
 
         this.notifiers.add(new Notifier() {
             @Override
@@ -121,15 +124,6 @@ public class BadQueryDetector extends Thread {
         ArrayList<Entry> entries = new ArrayList<Entry>(runningQueries.values());
         Collections.sort(entries);
         
-        // report if low memory
-        if (getSystemAvailMB() < alertMB) {
-            logger.info("System free memory less than " + alertMB + " MB. " + entries.size() + " queries running.");
-            for (int i = 0; i < entries.size(); i++) {
-                Entry e = entries.get(i);
-                notify("Low mem", (int) ((now - e.startTime) / 1000), e.sqlRequest.getSql());
-            }
-        }
-
         // report if query running long
         for (Entry e : entries) {
             int runningSec = (int) ((now - e.startTime) / 1000);
@@ -139,6 +133,27 @@ public class BadQueryDetector extends Thread {
                 break; // entries are sorted by startTime
             }
         }
+
+        // report if low memory
+        if (getSystemAvailMB() < alertMB) {
+            logger.info("System free memory less than " + alertMB + " MB. " + entries.size() + " queries running.");
+            
+            for (Map.Entry<Thread, Entry> mapEntry : runningQueries.entrySet()) {
+                Entry e = mapEntry.getValue();
+                int duration = (int) ((now - e.startTime) / 1000);
+                if (duration > killRunningSec) {
+                    notify("Kill", duration, e.sqlRequest.getSql());
+                    Thread queryThread = mapEntry.getKey();
+                    killQueryThread(queryThread);
+                } else {
+                    notify("Low mem", duration, e.sqlRequest.getSql());
+                }
+            }
+        }
+    }
+    
+    private void killQueryThread(Thread thread) {
+        thread.interrupt();
     }
 
     public static final int ONE_MB = 1024 * 1024;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c611d7ee/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java b/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java
index c849efd..d7e8ba9 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java
@@ -35,7 +35,7 @@ public class BadQueryDetectorTest {
         String mockSql = "select * from just_a_test";
         final ArrayList<String[]> alerts = new ArrayList<>();
 
-        BadQueryDetector badQueryDetector = new BadQueryDetector(alertRunningSec * 1000, alertMB, alertRunningSec);
+        BadQueryDetector badQueryDetector = new BadQueryDetector(alertRunningSec * 1000, alertMB, alertRunningSec, alertRunningSec * 5);
         badQueryDetector.registerNotifier(new BadQueryDetector.Notifier() {
             @Override
             public void badQueryFound(String adj, int runningSec, String sql) {
@@ -63,7 +63,7 @@ public class BadQueryDetectorTest {
         // first check founds Low mem
         assertArrayEquals(new String[] { "Low mem", mockSql }, alerts.get(0));
         // second check founds Low mem & Slow
-        assertArrayEquals(new String[] { "Low mem", mockSql }, alerts.get(1));
-        assertArrayEquals(new String[] { "Slow", mockSql }, alerts.get(2));
+        assertArrayEquals(new String[] { "Slow", mockSql }, alerts.get(1));
+        assertArrayEquals(new String[] { "Low mem", mockSql }, alerts.get(2));
     }
 }


[19/45] incubator-kylin git commit: exclude server/logs from RAT

Posted by sh...@apache.org.
exclude server/logs from RAT

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

Branch: refs/heads/master
Commit: 20f6eaf69fd1fa74c801d9df808bc4fc59e48fce
Parents: 734bf3a
Author: shaofengshi <sh...@apache.org>
Authored: Tue Sep 29 09:42:48 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 29 09:42:48 2015 +0800

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


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/20f6eaf6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 56ed278..e72c81f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -622,6 +622,7 @@
                                 <exclude>**/*.pdf</exclude>
                                 <exclude>**/*.log</exclude>
                                 <exclude>dist/**</exclude>
+                                <exclude>server/logs**</exclude>
 
                                 <exclude>**/.checkstyle</exclude>
                                 <!--Job's Test Data-->


[33/45] incubator-kylin git commit: minor, update how to contribute

Posted by sh...@apache.org.
minor, update how to contribute


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

Branch: refs/heads/master
Commit: 22917f53b5766d187b62d4ba03379f4936f9237f
Parents: 1c1a24c
Author: Yang Li <li...@apache.org>
Authored: Wed Oct 7 21:54:08 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Wed Oct 7 21:54:08 2015 +0800

----------------------------------------------------------------------
 website/_dev/howto_contribute.md | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/22917f53/website/_dev/howto_contribute.md
----------------------------------------------------------------------
diff --git a/website/_dev/howto_contribute.md b/website/_dev/howto_contribute.md
index 5e55438..6e87c0c 100644
--- a/website/_dev/howto_contribute.md
+++ b/website/_dev/howto_contribute.md
@@ -9,21 +9,20 @@ permalink: /development/howto_contribute.html
 
 Here are the major branches:
 
-* 0.7-staging: dev branch for 0.7 versions, this branch spawns releases like 0.7.1, 0.7.2, the next release would be 0.7.3.
-* 0.8: dev branch for 0.8 versions, 0.8 is like the next generation Kylin (with streaming, spark support), it has fundamental difference with 0.7 version, which means any changes on 0.7 will not be merged to 0.8 anymore. So if your patch affects both of the branch, you should make patches for both branch.
-* master: always point to the latest stable release(stable, but not up to date)
+* `1.x-staging`: Dev branch for 1,x versions, this branch spawns releases like 1.0, 1.1 etc. This is where new patches go to.
+* `2.x-staging`: Dev branch for 2.x versions, 2.x is the next generation Kylin (with streaming, spark support), it has fundamental difference with 1.x version, which means any changes on 1.x cannot merge to 2.x anymore. So if your patch affects both branches, you should make patches for both branches.
+* `master`: always point to the latest stable release (stable, but not up to date)
 
 ## Making Changes
 * Raise an issue on JIRA, describe the feature/enhancement/bug
 * Discuss with others in mailing list or issue comments, make sure the proposed changes fit in with what others are doing and have planned for the project
 * Make changes in your fork
 * Write unit test if no existing cover your change
-* Generate patch and attach it to relative JIRA; Please use "git format-patch" command to generate the patch, for a detail guide you can refer to [How to create and apply a patch with Git](https://ariejan.net/2009/10/26/how-to-create-and-apply-a-patch-with-git/)
+* Generate patch and attach it to relative JIRA; Please use `git format-patch` command to generate the patch, for a detail guide you can refer to [How to create and apply a patch with Git](https://ariejan.net/2009/10/26/how-to-create-and-apply-a-patch-with-git/)
 
 
 ## Apply Patch
 * Committer will review in terms of correctness, performance, design, coding style, test coverage
 * Discuss and revise if necessary
-* Finally committer merge code into main branch
-
+* Finally committer merge code into target branch
 


[16/45] incubator-kylin git commit: KYLIN-1048 fix CPU and memory killer in Cuboid.findById()

Posted by sh...@apache.org.
KYLIN-1048 fix CPU and memory killer in Cuboid.findById()

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

Branch: refs/heads/master
Commit: 6adf4a159f86272ce8194ebf758bd9a486033158
Parents: e82eb00
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 28 15:36:21 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 28 15:36:21 2015 +0800

----------------------------------------------------------------------
 cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/6adf4a15/cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java b/cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
index 9080115..5ef0c37 100644
--- a/cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
+++ b/cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
@@ -126,7 +126,7 @@ public class Cuboid implements Comparable<Cuboid> {
         long cuboidWithoutMandatory = cuboidID & ~rowkey.getMandatoryColumnMask();
         long leftover;
         for (AggrGroupMask mask : rowkey.getAggrGroupMasks()) {
-            if ((cuboidWithoutMandatory & mask.uniqueMask) == mask.uniqueMask) {
+            if ((cuboidWithoutMandatory & mask.uniqueMask) > 0) {
                 leftover = cuboidWithoutMandatory & ~mask.groupMask;
 
                 if (leftover == 0) {


[11/45] incubator-kylin git commit: Add blog for hybrid model

Posted by sh...@apache.org.
Add blog for hybrid model


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

Branch: refs/heads/master
Commit: be9c48464af29fff2f275e6d187d5ada018dc789
Parents: c611d7e
Author: shaofengshi <sh...@apache.org>
Authored: Fri Sep 25 23:30:54 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Sep 25 23:31:31 2015 +0800

----------------------------------------------------------------------
 website/_dev/dev_env.md                        |   5 +-
 website/_docs/index.md                         |   4 +-
 website/_posts/blog/2015-09-22-hybrid-model.md | 128 ++++++++++++++++++++
 website/images/blog/hybrid-model.png           | Bin 0 -> 118183 bytes
 4 files changed, 132 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/be9c4846/website/_dev/dev_env.md
----------------------------------------------------------------------
diff --git a/website/_dev/dev_env.md b/website/_dev/dev_env.md
index 20bb69f..a14fc2f 100644
--- a/website/_dev/dev_env.md
+++ b/website/_dev/dev_env.md
@@ -75,8 +75,9 @@ Run a end-to-end cube building test, these special test cases will populate some
 It might take a while (maybe one hour), please keep patient.
  
 {% highlight Groff markup %}
-	mvn test -Dtest=org.apache.kylin.job.BuildCubeWithEngineTest -DfailIfNoTests=false -P sandbox
-	mvn test -Dtest=org.apache.kylin.job.BuildIIWithEngineTest -DfailIfNoTests=false -P sandbox
+	mvn test -Dtest=org.apache.kylin.job.BuildCubeWithEngineTest -DfailIfNoTests=false -Dhdp.version=<hdp-version> -P sandbox
+	
+	mvn test -Dtest=org.apache.kylin.job.BuildIIWithEngineTest -DfailIfNoTests=false -Dhdp.version=<hdp-version> -P sandbox
 {% endhighlight %}
 	
 Run other tests, the end-to-end cube building test is exclueded

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/be9c4846/website/_docs/index.md
----------------------------------------------------------------------
diff --git a/website/_docs/index.md b/website/_docs/index.md
index 64e1295..c91054e 100644
--- a/website/_docs/index.md
+++ b/website/_docs/index.md
@@ -41,9 +41,7 @@ Advanced Topics
 
 1.[Check Kylin Metadata Store](howto/howto_backup_metadata.html)
 
-2.[Clean/Export Kylin HBase data](howto/howto_backup.html)
-
-3.[Advanced settings of Kylin environment](install/advance_settings.html)
+2.[Advanced settings of Kylin environment](install/advance_settings.html)
 
 ---
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/be9c4846/website/_posts/blog/2015-09-22-hybrid-model.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-09-22-hybrid-model.md b/website/_posts/blog/2015-09-22-hybrid-model.md
new file mode 100644
index 0000000..1e0f4c7
--- /dev/null
+++ b/website/_posts/blog/2015-09-22-hybrid-model.md
@@ -0,0 +1,128 @@
+---
+layout: post-blog
+title:  "Hybrid Model in Apache Kylin 1.0"
+date:   2015-09-25 16:00:00
+author: Shaofeng Shi
+categories: blog
+---
+
+**Apache Kylin v1.0 introduces a new realization "hybrid model" (also called "dynamic model"); This post introduces the concept and how to define it.**
+
+# Problem
+
+For incoming SQL queries, Kylin picks ONE (and only ONE) realization to serve the query; Before the "hybrid", there is only one type of realization open for user: Cube. That to say, only 1 Cube would be selected to answer a query;
+
+Now let's start with a sample case; Assume user has a Cube called "Cube_V1", it has been built for a couple of months; Now the user wants to add new dimension or metrics to fulfill their business need; So he created a new Cube named "Cube_V2"; 
+
+Due to some reason user wants to keep the data of "Cube_V1", and expects to build "Cube_V2" from the end date of "Cube_V1"; The possible reasons include:
+
+* History source data has been dropped from Hadoop, not possible to build "Cube_V2" from the very beginning;
+* The cube is large, rebuilding takes very long time;
+* New dimension/metrics is only feasible for the new date, or user feels fine if they were absent for old cube; etc.
+
+For some queries that don't use the new measure and metrics, user hopes both "Cube_V1" and "Cube_V2" can be scanned to get a full result, such as "select count(*)...", "select sum(price)..."; With such a background, the "hybrid model" is introduced in Kylin;
+
+## Hybrid Model
+
+Hybrid model is a new realization which is a composite of one or multiple other realizations (cubes); See the figure below.
+
+![]( /images/blog/hybrid-model.png)
+
+Hybrid doesn't have its real storage; It is just like a virtual database view over tables; It acts as a delegator who delegates the requests to its children realizations.
+
+## How to add a Hybrid model
+
+As there is no UI for creating/editing hybrid model, if have the need, you need manually edit Kylin metadata;
+
+### Step 1: Take a backup of kylin metadata store 
+
+```
+export KYLIN_HOME="/path/to/kylin"
+
+$KYLIN_HOME/bin/metastore.sh backup
+
+```
+
+A backup folder will be created, assume it is $KYLIN_HOME/metadata_backup/2015-09-25/
+ 
+### Step 2: Create sub-folder "hybrid" in the metadata folder,
+
+```
+mkdir -p $KYLIN_HOME/metadata_backup/2015-09-25/hybrid
+```
+
+### Step 3: Create a hybrid json file: 
+
+```
+vi $KYLIN_HOME/metadata_backup/2015-09-25/hybrid/my_hybrid.json
+
+```
+
+Input content like this:
+
+```
+{
+  "uuid": "9iiu8590-64b6-4367-8fb5-7500eb95fd9c",
+  "name": "my_hybrid",
+  "realizations": [
+    {
+           "type": "CUBE",
+           "realization": "Cube_V1"
+    },
+    {
+            "type": "CUBE",
+            "realization": "Cube_V2"
+    }
+  ]
+}
+
+```
+Here "Cube_V1" and "Cube_V2" are the cubes that you want to combine.
+
+
+### Step 4: Add hybrid model to project
+
+Open project json file (for example project "default") with text editor:
+
+```
+vi $KYLIN_HOME/metadata_backup/2015-09-25/project/default.json
+
+```
+
+In the "realizations" array, add one entry like:
+
+```
+    {
+      "name": "my_hybrid",
+      "type": "HYBRID",
+      "realization": "my_hybrid"
+    }
+```
+
+### Step 5: Upload the metadata:
+
+```
+  $KYLIN_HOME/bin/metastore.sh restore $KYLIN_HOME/metadata_backup/2015-09-25/
+
+```
+
+### Step 6: Reload metadata
+
+Restart Kylin server, or click "Reload metadata" in the "Admin" tab on Kylin web UI to load the changes; Ideally the hybrid will start to work; You can do some verifications.
+
+## FAQ:
+
+**Question 1**: when will hybrid be selected to serve query?
+If one of the cube can answer the query, the hybrid which has it as a child will be selected;
+
+**Question 2**: how hybrid to answer the query?
+Hybrid will delegate the query to each of its child realization (if it is capable); And then return all the results to query engine; Query engine will aggregate before return to user;
+
+**Question 3**: will hybrid check the data duplication?
+No; it depends on you to ensure the cubes in a hybrid don't have date/time range duplication; For example, the "Cube_V1" is ended at 2015-9-20 (including), the "Cube_V2" should start from 2015-9-21 or later;
+
+**Question 4**: will hybrid restrict the children cubes having the same data model?
+No; hybrid doesn't check the cube's fact/lookup tables and join conditions at all;
+
+**Question 5**: can hybrid have another hybrid as child?
+No; didn't see the need; so far it assumes all children are Cubes;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/be9c4846/website/images/blog/hybrid-model.png
----------------------------------------------------------------------
diff --git a/website/images/blog/hybrid-model.png b/website/images/blog/hybrid-model.png
new file mode 100644
index 0000000..5fd476c
Binary files /dev/null and b/website/images/blog/hybrid-model.png differ


[21/45] incubator-kylin git commit: KYLIN-742 Route unsupported queries to Hive

Posted by sh...@apache.org.
KYLIN-742 Route unsupported queries to Hive

Signed-off-by: Li, Yang <ya...@ebay.com>


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

Branch: refs/heads/master
Commit: 7990438023e898f1f01a62185f5607b72ef82643
Parents: 6732263
Author: murkrishn <mu...@ebay.com>
Authored: Mon Sep 14 23:51:39 2015 -0700
Committer: Li, Yang <ya...@ebay.com>
Committed: Tue Sep 29 15:40:26 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/KylinConfig.java    |  27 ++
 .../test_case_data/sandbox/kylin.properties     | 278 ++++++++++---------
 .../apache/kylin/rest/service/QueryService.java |  29 +-
 .../apache/kylin/rest/util/HiveRerouteUtil.java | 133 +++++++++
 4 files changed, 330 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/79904380/common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/KylinConfig.java b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
index d3220ee..72b6a65 100644
--- a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -132,6 +132,15 @@ public class KylinConfig {
 
     public static final String HBASE_REGION_COUNT_MIN = "kylin.hbase.region.count.min";
     public static final String HBASE_REGION_COUNT_MAX = "kylin.hbase.region.count.max";
+
+    // property for Hive query rerouting enablement
+    public static final String KYLIN_ROUTE_HIVE_ENABLED = "kylin.route.hive.enabled";
+    public static final boolean KYLIN_ROUTE_HIVE_ENABLED_DEFAULT = false;
+    // JDBC Hive connection details for query rerouting
+    public static final String KYLIN_ROUTE_HIVE_URL = "kylin.route.hive.url";
+    public static final String KYLIN_ROUTE_HIVE_URL_DEFAULT = "jdbc:hive2://sandbox:10000";
+    public static final String KYLIN_ROUTE_HIVE_USERNAME = "kylin.route.hive.username";
+    public static final String KYLIN_ROUTE_HIVE_PASSWORD = "kylin.route.hive.password";
     
     // static cached instances
     private static KylinConfig ENV_INSTANCE = null;
@@ -265,6 +274,24 @@ public class KylinConfig {
 
     // ============================================================================
 
+    // start: properties for Hive rerouting
+    public boolean isHiveReroutingEnabled() {
+	return Boolean.parseBoolean(getOptional(KYLIN_ROUTE_HIVE_ENABLED)); 
+    }
+
+    public String getHiveRerouteUrl() {
+        return getOptional(KYLIN_ROUTE_HIVE_URL, KYLIN_ROUTE_HIVE_URL_DEFAULT);
+    }
+
+    public String getHiveRerouteUsername() {
+        return getOptional(KYLIN_ROUTE_HIVE_USERNAME, "");
+    }
+
+    public String getHiveReroutePassword() {
+        return getOptional(KYLIN_ROUTE_HIVE_PASSWORD, "");
+    }
+    // end: properties for JDBC Hive rerouting
+
     public String getStorageUrl() {
         return storageUrl;
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/79904380/examples/test_case_data/sandbox/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin.properties b/examples/test_case_data/sandbox/kylin.properties
index bcae574..6fe63a8 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -1,136 +1,142 @@
-#
-# 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.
-#
-
-## Config for Kylin Engine ##
-
-# List of web servers in use, this enables one web server instance to sync up with other servers.
-kylin.rest.servers=localhost:7070
-
-#set display timezone on UI,format like[GMT+N or GMT-N]
-kylin.rest.timezone=GMT-8
-
-# The metadata store in hbase
-kylin.metadata.url=kylin_metadata@hbase
-
-# The storage for final cube file in hbase
-kylin.storage.url=hbase
-
-# Temp folder in hdfs, make sure user has the right access to the hdfs directory
-kylin.hdfs.working.dir=/kylin
-
-kylin.job.mapreduce.default.reduce.input.mb=500
-
-# If true, job engine will not assume that hadoop CLI reside on the same server as it self
-# you will have to specify kylin.job.remote.cli.hostname, kylin.job.remote.cli.username and kylin.job.remote.cli.password
-kylin.job.run.as.remote.cmd=false
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.hostname=
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.username=
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.password=
-
-# Used by test cases to prepare synthetic data for sample cube
-kylin.job.remote.cli.working.dir=/tmp/kylin
-
-# Max count of concurrent jobs running
-kylin.job.concurrent.max.limit=10
-
-# Time interval to check hadoop job status
-kylin.job.yarn.app.rest.check.interval.seconds=10
-
-# Hive database name for putting the intermediate flat tables
-kylin.job.hive.database.for.intermediatetable=default
-
-#default compression codec for htable,snappy,lzo,gzip,lz4
-kylin.hbase.default.compression.codec=gzip
-
-# The cut size for hbase region, in GB.
-# E.g, for cube whose capacity be marked as "SMALL", split region per 10GB by default
-kylin.hbase.region.cut.small=10
-kylin.hbase.region.cut.medium=20
-kylin.hbase.region.cut.large=100
-
-# HBase min and max region count
-kylin.hbase.region.count.min=1
-kylin.hbase.region.count.min=500
-
-## Config for Restful APP ##
-# database connection settings:
-ldap.server=
-ldap.username=
-ldap.password=
-ldap.user.searchBase=
-ldap.user.searchPattern=
-ldap.user.groupSearchBase=
-ldap.service.searchBase=OU=
-ldap.service.searchPattern=
-ldap.service.groupSearchBase=
-acl.adminRole=
-acl.defaultRole=
-ganglia.group=
-ganglia.port=8664
-
-## Config for mail service
-
-# If true, will send email notification;
-mail.enabled=false
-mail.host=
-mail.username=
-mail.password=
-mail.sender=
-
-###########################config info for web#######################
-
-#help info ,format{name|displayName|link} ,optional
-kylin.web.help.length=4
-kylin.web.help.0=start|Getting Started|
-kylin.web.help.1=odbc|ODBC Driver|
-kylin.web.help.2=tableau|Tableau Guide|
-kylin.web.help.3=onboard|Cube Design Tutorial|
-#hadoop url link ,optional
-kylin.web.hadoop=
-#job diagnostic url link ,optional
-kylin.web.diagnostic=
-#contact mail on web page ,optional
-kylin.web.contact_mail=
-
-###########################config info for front#######################
-
-#env DEV|QA|PROD
-deploy.env=DEV
-
-###########################config info for sandbox#######################
-kylin.sandbox=true
-
-
-###########################config info for kylin monitor#######################
-# hive jdbc url
-kylin.monitor.hive.jdbc.connection.url= jdbc:hive2://sandbox:10000
-
-#config where to parse query log,split with comma ,will also read $KYLIN_HOME/tomcat/logs/ by default
-kylin.monitor.ext.log.base.dir = /tmp/kylin_log1,/tmp/kylin_log2
-
-#will create external hive table to query result csv file
-#will set to kylin_query_log by default if not config here
-kylin.monitor.query.log.parse.result.table = kylin_query_log
-
-
-
+#
+# 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.
+#
+
+## Config for Kylin Engine ##
+
+# List of web servers in use, this enables one web server instance to sync up with other servers.
+kylin.rest.servers=localhost:7070
+
+#set display timezone on UI,format like[GMT+N or GMT-N]
+kylin.rest.timezone=GMT-8
+
+# The metadata store in hbase
+kylin.metadata.url=kylin_metadata@hbase
+
+# The storage for final cube file in hbase
+kylin.storage.url=hbase
+
+# Temp folder in hdfs, make sure user has the right access to the hdfs directory
+kylin.hdfs.working.dir=/kylin
+
+kylin.job.mapreduce.default.reduce.input.mb=500
+
+# If true, job engine will not assume that hadoop CLI reside on the same server as it self
+# you will have to specify kylin.job.remote.cli.hostname, kylin.job.remote.cli.username and kylin.job.remote.cli.password
+kylin.job.run.as.remote.cmd=false
+
+# Only necessary when kylin.job.run.as.remote.cmd=true
+kylin.job.remote.cli.hostname=
+
+# Only necessary when kylin.job.run.as.remote.cmd=true
+kylin.job.remote.cli.username=
+
+# Only necessary when kylin.job.run.as.remote.cmd=true
+kylin.job.remote.cli.password=
+
+# Used by test cases to prepare synthetic data for sample cube
+kylin.job.remote.cli.working.dir=/tmp/kylin
+
+# Max count of concurrent jobs running
+kylin.job.concurrent.max.limit=10
+
+# Time interval to check hadoop job status
+kylin.job.yarn.app.rest.check.interval.seconds=10
+
+# Hive database name for putting the intermediate flat tables
+kylin.job.hive.database.for.intermediatetable=default
+
+#default compression codec for htable,snappy,lzo,gzip,lz4
+kylin.hbase.default.compression.codec=gzip
+
+# The cut size for hbase region, in GB.
+# E.g, for cube whose capacity be marked as "SMALL", split region per 10GB by default
+kylin.hbase.region.cut.small=10
+kylin.hbase.region.cut.medium=20
+kylin.hbase.region.cut.large=100
+
+# HBase min and max region count
+kylin.hbase.region.count.min=1
+kylin.hbase.region.count.min=500
+
+## Config for Restful APP ##
+# database connection settings:
+ldap.server=
+ldap.username=
+ldap.password=
+ldap.user.searchBase=
+ldap.user.searchPattern=
+ldap.user.groupSearchBase=
+ldap.service.searchBase=OU=
+ldap.service.searchPattern=
+ldap.service.groupSearchBase=
+acl.adminRole=
+acl.defaultRole=
+ganglia.group=
+ganglia.port=8664
+
+## Config for mail service
+
+# If true, will send email notification;
+mail.enabled=false
+mail.host=
+mail.username=
+mail.password=
+mail.sender=
+
+###########################config info for web#######################
+
+#help info ,format{name|displayName|link} ,optional
+kylin.web.help.length=4
+kylin.web.help.0=start|Getting Started|
+kylin.web.help.1=odbc|ODBC Driver|
+kylin.web.help.2=tableau|Tableau Guide|
+kylin.web.help.3=onboard|Cube Design Tutorial|
+#hadoop url link ,optional
+kylin.web.hadoop=
+#job diagnostic url link ,optional
+kylin.web.diagnostic=
+#contact mail on web page ,optional
+kylin.web.contact_mail=
+
+###########################config info for front#######################
+
+#env DEV|QA|PROD
+deploy.env=DEV
+
+###########################config info for sandbox#######################
+kylin.sandbox=true
+
+
+###########################config info for kylin monitor#######################
+# hive jdbc url
+kylin.monitor.hive.jdbc.connection.url= jdbc:hive2://sandbox:10000
+
+#config where to parse query log,split with comma ,will also read $KYLIN_HOME/tomcat/logs/ by default
+kylin.monitor.ext.log.base.dir = /tmp/kylin_log1,/tmp/kylin_log2
+
+#will create external hive table to query result csv file
+#will set to kylin_query_log by default if not config here
+kylin.monitor.query.log.parse.result.table = kylin_query_log
+
+##### Config for enabling query rerouting to Hive. Disabled by default. ######
+kylin.route.hive.enabled=false
+
+# If query rerouting is enabled, provide the hive configurations
+# Default value for kylin.route.hive.url will be pointing to the embedded server (jdbc:hive2://)
+kylin.route.hive.url=
+kylin.route.hive.username=
+kylin.route.hive.password=

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/79904380/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 6e6d9da..81905c8 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -40,6 +40,9 @@ import java.util.Set;
 
 import javax.sql.DataSource;
 
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.calcite.sql.parser.impl.ParseException;
+import org.apache.calcite.sql.validate.SqlValidatorException;
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.client.Get;
@@ -62,6 +65,7 @@ import org.apache.kylin.rest.model.TableMeta;
 import org.apache.kylin.rest.request.PrepareSqlRequest;
 import org.apache.kylin.rest.request.SQLRequest;
 import org.apache.kylin.rest.response.SQLResponse;
+import org.apache.kylin.rest.util.HiveRerouteUtil;
 import org.apache.kylin.rest.util.QueryUtil;
 import org.apache.kylin.rest.util.Serializer;
 import org.slf4j.Logger;
@@ -372,7 +376,30 @@ public class QueryService extends BasicService {
                 results.add(new LinkedList<String>(oneRow));
                 oneRow.clear();
             }
-        } finally {
+        } catch (SQLException sqlException) {
+            // unsuccessful statement execution, retry with Hive on Spark. Code modification as part of the jira https://issues.apache.org/jira/browse/KYLIN-742
+	    boolean isExpectedCause = (ExceptionUtils.getRootCause(sqlException).getClass().equals(SqlValidatorException.class)) || (ExceptionUtils.getRootCause(sqlException).getClass().equals(ParseException.class));
+            KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+    	    if (isExpectedCause && kylinConfig.isHiveReroutingEnabled()) {
+        	logger.debug("query rerouting option enabled for Kylin");
+        	// running query to hive
+                HiveRerouteUtil rerouteUtil = new HiveRerouteUtil();
+                try {
+                    conn = rerouteUtil.createConnection(kylinConfig.getHiveRerouteUrl(), kylinConfig.getHiveRerouteUsername(), kylinConfig.getHiveReroutePassword());
+                    resultSet = rerouteUtil.executQuery(conn, sql);
+                    columnMetas = rerouteUtil.extractColumnMetadata(resultSet, columnMetas);
+                    results = rerouteUtil.extractResults(resultSet, results);
+                } catch (Exception exception) {
+                    logger.error("exception in re-routing the query to hive", exception);
+                    throw exception;
+                } finally {
+                    rerouteUtil.closeConnection(conn);
+                } 
+            } else {
+                logger.error("exception in running the query: " + sql);
+		throw sqlException;
+    	    }
+	    } finally {
             close(resultSet, stat, conn);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/79904380/server/src/main/java/org/apache/kylin/rest/util/HiveRerouteUtil.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/util/HiveRerouteUtil.java b/server/src/main/java/org/apache/kylin/rest/util/HiveRerouteUtil.java
new file mode 100644
index 0000000..ca1e048
--- /dev/null
+++ b/server/src/main/java/org/apache/kylin/rest/util/HiveRerouteUtil.java
@@ -0,0 +1,133 @@
+/*
+ * 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.kylin.rest.util;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.kylin.rest.model.SelectedColumnMeta;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** @author murkrishn **/
+public class HiveRerouteUtil {
+
+    private static final Logger logger = LoggerFactory.getLogger(HiveRerouteUtil.class);
+    public static final String driverName = "org.apache.hive.jdbc.HiveDriver";
+
+    /**
+     * Create a connection to the Hive server by passing the required connection parameters.
+     * @param connectionURL: JDBC URL to connect to the Hive server
+     * @param username: Username to connect with (optional)
+     * @param password: Password to connect with (optional)
+     * @return: Connection object to the Hive server
+     * @throws Exception
+     */
+    public Connection createConnection (String connectionURL, String username, String password) throws Exception {
+        logger.info("rerouting to : " + connectionURL + " for executing the query");
+        
+        try {
+            Class.forName(driverName);
+        } catch (ClassNotFoundException classNotFoundException) {
+            throw classNotFoundException;
+        }
+        
+        Connection connection = DriverManager.getConnection(connectionURL, username, password);
+        return connection;
+    }
+    
+    /**
+     * Close the connection to the Hive server.
+     * @param connection: Connection object to be closed
+     */
+    public void closeConnection(Connection connection) {
+        if (null != connection) {
+            try {
+                connection.close();
+            } catch (SQLException sqlException) {
+                logger.error("failed to close connection", sqlException);
+            }
+        }
+    }
+    
+    /**
+     * Execute a query in Hive.
+     * @param connection: Connection object to the Hive server
+     * @param query: Query to be executed
+     * @return: ResultSet object of the query executed
+     * @throws Exception
+     */
+    public ResultSet executQuery (Connection connection, String query) throws Exception {
+        Statement statement = null;
+        ResultSet resultSet = null;
+        
+        try {
+            statement = connection.createStatement();
+            resultSet = statement.executeQuery(query);
+            return resultSet;
+        } catch (SQLException sqlException) {
+            throw sqlException;
+        }
+    }
+    
+    public List<SelectedColumnMeta> extractColumnMetadata (ResultSet resultSet, List<SelectedColumnMeta> columnMetas) throws SQLException {
+        ResultSetMetaData metaData = null;
+        int columnCount = 0;
+        
+        try {
+            metaData = resultSet.getMetaData();
+            columnCount = metaData.getColumnCount();
+            
+            // fill in selected column meta
+            for (int i = 1; i <= columnCount; ++i) {
+                columnMetas.add(new SelectedColumnMeta(metaData.isAutoIncrement(i), metaData.isCaseSensitive(i), false, metaData.isCurrency(i), metaData.isNullable(i), false, metaData.getColumnDisplaySize(i), metaData.getColumnLabel(i), metaData.getColumnName(i), null, null, null, metaData.getPrecision(i), metaData.getScale(i), metaData.getColumnType(i), metaData.getColumnTypeName(i), metaData.isReadOnly(i), false, false));
+            }
+            
+            return columnMetas;
+        } catch (SQLException sqlException) {
+            throw sqlException;
+        }
+    }
+    
+    public List<List<String>> extractResults (ResultSet resultSet, List<List<String>> results) throws SQLException {
+        List<String> oneRow = new LinkedList<String>();
+        
+        try {
+            while (resultSet.next()) {
+                //logger.debug("resultSet value: " + resultSet.getString(1));
+                for (int i = 0; i < resultSet.getMetaData().getColumnCount(); i++) {
+                    oneRow.add((resultSet.getString(i + 1)));
+                }
+
+                results.add(new LinkedList<String>(oneRow));
+                oneRow.clear();
+            }
+            
+            return results;
+        } catch (SQLException sqlException) {
+            throw sqlException;
+        }
+    }
+}
\ No newline at end of file


[09/45] incubator-kylin git commit: minor, pass RAT check for II.proto and others

Posted by sh...@apache.org.
minor, pass RAT check for II.proto and others


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

Branch: refs/heads/master
Commit: 62751df48ff51621986b6fd6a661f1f7a81e3c98
Parents: ba57407
Author: Li, Yang <ya...@ebay.com>
Authored: Fri Sep 25 10:33:55 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Fri Sep 25 10:33:55 2015 +0800

----------------------------------------------------------------------
 pom.xml                                            |  2 +-
 .../hbase/coprocessor/endpoint/protobuf/II.proto   | 17 +++++++++++++++++
 2 files changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/62751df4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1f82807..ae9714c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -624,7 +624,7 @@
 
                                 <exclude>**/.checkstyle</exclude>
                                 <!--Job's Test Data-->
-                                <exclude>**/src/test/resources/data/**</exclude>
+                                <exclude>**/src/test/resources/**</exclude>
 
                                 <!-- generated files -->
                                 <exclude>**/target/**</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/62751df4/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/protobuf/II.proto
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/protobuf/II.proto b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/protobuf/II.proto
index 18490fa..9b34141 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/protobuf/II.proto
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/protobuf/II.proto
@@ -1,3 +1,20 @@
+//
+// 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.
+//
 
 // usage:
 // protoc  --java_out=./storage/src/main/java  ./storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/protobuf/II.proto


[13/45] incubator-kylin git commit: KYLIN-1039 Fix un-evaluatable in LogicalTupleFilter

Posted by sh...@apache.org.
KYLIN-1039 Fix un-evaluatable in LogicalTupleFilter


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

Branch: refs/heads/master
Commit: 38f4fd01f47b37357a0f98501e5f71a31aaaa268
Parents: be9c484
Author: Yang Li <li...@apache.org>
Authored: Sat Sep 26 09:56:05 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sat Sep 26 09:56:05 2015 +0800

----------------------------------------------------------------------
 .../metadata/filter/LogicalTupleFilter.java     | 22 +++++++++++++++++++-
 .../hbase/coprocessor/CoprocessorFilter.java    |  8 +++----
 2 files changed, 24 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/38f4fd01/metadata/src/main/java/org/apache/kylin/metadata/filter/LogicalTupleFilter.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/filter/LogicalTupleFilter.java b/metadata/src/main/java/org/apache/kylin/metadata/filter/LogicalTupleFilter.java
index c17310a..1421440 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/filter/LogicalTupleFilter.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/filter/LogicalTupleFilter.java
@@ -112,7 +112,27 @@ public class LogicalTupleFilter extends TupleFilter {
 
     @Override
     public boolean isEvaluable() {
-        return true;
+        switch (operator) {
+        case NOT:
+            // Un-evaluatable branch will be pruned and be replaced with TRUE.
+            // And this must happen at the top NOT, otherwise NOT (TRUE) becomes false.
+            for (TupleFilter child : children) {
+                if (TupleFilter.isEvaluableRecursively(child) == false)
+                    return false;
+            }
+            return true;
+        case OR:
+            // (anything OR un-evaluable) will become (anything or TRUE) which is effectively TRUE.
+            // The "anything" is not evaluated, kinda disabled, by the un-evaluable part.
+            // If it's partially un-evaluable, then "anything" is partially disabled, and the OR is still not fully evaluatable.
+            for (TupleFilter child : children) {
+                if (TupleFilter.isEvaluableRecursively(child) == false)
+                    return false;
+            }
+            return true;
+        default:
+            return true;
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/38f4fd01/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorFilter.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorFilter.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorFilter.java
index 70d59ca..65fddd2 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorFilter.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/CoprocessorFilter.java
@@ -30,7 +30,6 @@ import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
 import org.apache.kylin.metadata.filter.TupleFilterSerializer;
 import org.apache.kylin.metadata.filter.TupleFilterSerializer.Decorator;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -62,10 +61,8 @@ public class CoprocessorFilter {
             if (filter == null)
                 return null;
 
-            // In case of NOT(unEvaluatableFilter), we should immediatedly replace it as TRUE,
-            // Otherwise, unEvaluatableFilter will later be replace with TRUE and NOT(unEvaluatableFilter) will
-            // always return FALSE
-            if (filter.getOperator() == FilterOperatorEnum.NOT && !TupleFilter.isEvaluableRecursively(filter)) {
+            // un-evaluatable filter is replaced with TRUE
+            if (!filter.isEvaluable()) {
                 TupleFilter.collectColumns(filter, unstrictlyFilteredColumns);
                 return ConstantTupleFilter.TRUE;
             }
@@ -73,6 +70,7 @@ public class CoprocessorFilter {
             if (!(filter instanceof CompareTupleFilter))
                 return filter;
 
+            // double check all internal of CompareTupleFilter is evaluatable
             if (!TupleFilter.isEvaluableRecursively(filter)) {
                 TupleFilter.collectColumns(filter, unstrictlyFilteredColumns);
                 return ConstantTupleFilter.TRUE;


[32/45] incubator-kylin git commit: minor, fix NPE in CubingJob.formatNotifications()

Posted by sh...@apache.org.
minor, fix NPE in CubingJob.formatNotifications()


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

Branch: refs/heads/master
Commit: 1c1a24cdd5dba36e27b9419efcc975ab721836cd
Parents: f00c838
Author: Yang Li <li...@apache.org>
Authored: Wed Oct 7 09:27:43 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Wed Oct 7 09:27:43 2015 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/kylin/common/util/StringUtil.java   | 8 +++++++-
 job/src/main/java/org/apache/kylin/job/cube/CubingJob.java   | 5 +++--
 2 files changed, 10 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1c1a24cd/common/src/main/java/org/apache/kylin/common/util/StringUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/util/StringUtil.java b/common/src/main/java/org/apache/kylin/common/util/StringUtil.java
index e500612..dbf3380 100644
--- a/common/src/main/java/org/apache/kylin/common/util/StringUtil.java
+++ b/common/src/main/java/org/apache/kylin/common/util/StringUtil.java
@@ -21,6 +21,8 @@ package org.apache.kylin.common.util;
 import java.util.ArrayList;
 import java.util.Collection;
 
+import org.apache.commons.lang.StringUtils;
+
 /**
  * Created with IntelliJ IDEA. User: lukhan Date: 12/2/13 Time: 11:43 AM To
  * change this template use File | Settings | File Templates.
@@ -56,7 +58,11 @@ public class StringUtil {
             }
         }
     }
-
+    
+    public static String noBlank(String str, String dft) {
+        return StringUtils.isBlank(str) ? dft : str;
+    }
+    
     public static String dropSuffix(String str, String suffix) {
         if (str.endsWith(suffix))
             return str.substring(0, str.length() - suffix.length());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1c1a24cd/job/src/main/java/org/apache/kylin/job/cube/CubingJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/CubingJob.java b/job/src/main/java/org/apache/kylin/job/cube/CubingJob.java
index bf491a8..760a6c3 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/CubingJob.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/CubingJob.java
@@ -25,6 +25,7 @@ import java.util.List;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.job.common.MapReduceExecutable;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.execution.AbstractExecutable;
@@ -91,8 +92,8 @@ public class CubingJob extends DefaultChainedExecutable {
         content = content.replaceAll("\\$\\{duration\\}", getDuration() / 60000 + "mins");
         content = content.replaceAll("\\$\\{mr_waiting\\}", getMapReduceWaitTime() / 60000 + "mins");
         content = content.replaceAll("\\$\\{last_update_time\\}", new Date(getLastModified()).toString());
-        content = content.replaceAll("\\$\\{submitter\\}", getSubmitter());
-        content = content.replaceAll("\\$\\{error_log\\}", logMsg);
+        content = content.replaceAll("\\$\\{submitter\\}", StringUtil.noBlank(getSubmitter(), "missing submitter"));
+        content = content.replaceAll("\\$\\{error_log\\}", StringUtil.noBlank(logMsg, "no error log"));
 
         try {
             InetAddress inetAddress = InetAddress.getLocalHost();


[22/45] incubator-kylin git commit: KYLIN-742 Minor refactoring

Posted by sh...@apache.org.
KYLIN-742 Minor refactoring


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

Branch: refs/heads/master
Commit: 143aa608b88c0ceadeb11b38bf6b2dbaee105c5e
Parents: 7990438
Author: Li, Yang <ya...@ebay.com>
Authored: Tue Sep 29 17:41:53 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Tue Sep 29 17:42:35 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/KylinConfig.java    |  43 +++---
 .../test_case_data/sandbox/kylin.properties     |  17 ++-
 server/pom.xml                                  |  11 +-
 .../apache/kylin/rest/service/QueryService.java |  48 +++----
 .../org/apache/kylin/rest/util/HiveReroute.java | 142 +++++++++++++++++++
 .../apache/kylin/rest/util/HiveRerouteUtil.java | 133 -----------------
 6 files changed, 195 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/143aa608/common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/KylinConfig.java b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
index 72b6a65..14c0fb4 100644
--- a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -44,6 +44,13 @@ import com.google.common.collect.Sets;
  * @author yangli9
  */
 public class KylinConfig {
+    
+    /*
+     * NO NEED TO DEFINE PUBLIC CONSTANTS FOR KEY NAMES!
+     * 
+     * Kylin configuration is exposed as public methods. A client never need to access key names directly.
+     * Feel free to hard code key names, as long as they are encapsulated in this class. It reads better!
+     */
 
     public static final String KYLIN_STORAGE_URL = "kylin.storage.url";
 
@@ -96,11 +103,6 @@ public class KylinConfig {
 
     public static final String KYLIN_HBASE_CLUSTER_FS = "kylin.hbase.cluster.fs";
 
-    public static final String HIVE_PASSWORD = "hive.password";
-
-    public static final String HIVE_USER = "hive.user";
-
-    public static final String HIVE_URL = "hive.url";
     /**
      * Kylin properties file
      */
@@ -133,15 +135,6 @@ public class KylinConfig {
     public static final String HBASE_REGION_COUNT_MIN = "kylin.hbase.region.count.min";
     public static final String HBASE_REGION_COUNT_MAX = "kylin.hbase.region.count.max";
 
-    // property for Hive query rerouting enablement
-    public static final String KYLIN_ROUTE_HIVE_ENABLED = "kylin.route.hive.enabled";
-    public static final boolean KYLIN_ROUTE_HIVE_ENABLED_DEFAULT = false;
-    // JDBC Hive connection details for query rerouting
-    public static final String KYLIN_ROUTE_HIVE_URL = "kylin.route.hive.url";
-    public static final String KYLIN_ROUTE_HIVE_URL_DEFAULT = "jdbc:hive2://sandbox:10000";
-    public static final String KYLIN_ROUTE_HIVE_USERNAME = "kylin.route.hive.username";
-    public static final String KYLIN_ROUTE_HIVE_PASSWORD = "kylin.route.hive.password";
-    
     // static cached instances
     private static KylinConfig ENV_INSTANCE = null;
 
@@ -274,38 +267,42 @@ public class KylinConfig {
 
     // ============================================================================
 
-    // start: properties for Hive rerouting
     public boolean isHiveReroutingEnabled() {
-	return Boolean.parseBoolean(getOptional(KYLIN_ROUTE_HIVE_ENABLED)); 
+	return Boolean.parseBoolean(getOptional("kylin.route.hive.enabled", "false")); 
     }
 
     public String getHiveRerouteUrl() {
-        return getOptional(KYLIN_ROUTE_HIVE_URL, KYLIN_ROUTE_HIVE_URL_DEFAULT);
+        return getOptional("kylin.route.hive.url", "jdbc:hive2://");
     }
 
     public String getHiveRerouteUsername() {
-        return getOptional(KYLIN_ROUTE_HIVE_USERNAME, "");
+        return getOptional("kylin.route.hive.username", "");
     }
 
     public String getHiveReroutePassword() {
-        return getOptional(KYLIN_ROUTE_HIVE_PASSWORD, "");
+        return getOptional("kylin.route.hive.password", "");
     }
-    // end: properties for JDBC Hive rerouting
 
     public String getStorageUrl() {
         return storageUrl;
     }
 
+    /** Use the hive reroute feature instead */
+    @Deprecated
     public String getHiveUrl() {
-        return getOptional(HIVE_URL, "");
+        return getOptional("hive.url", "");
     }
 
+    /** Use the hive reroute feature instead */
+    @Deprecated
     public String getHiveUser() {
-        return getOptional(HIVE_USER, "");
+        return getOptional("hive.user", "");
     }
 
+    /** Use the hive reroute feature instead */
+    @Deprecated
     public String getHivePassword() {
-        return getOptional(HIVE_PASSWORD, "");
+        return getOptional("hive.password", "");
     }
 
     public String getHdfsWorkingDirectory() {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/143aa608/examples/test_case_data/sandbox/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin.properties b/examples/test_case_data/sandbox/kylin.properties
index 6fe63a8..fa1e4de 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -73,6 +73,7 @@ kylin.hbase.region.count.min=1
 kylin.hbase.region.count.min=500
 
 ## Config for Restful APP ##
+
 # database connection settings:
 ldap.server=
 ldap.username=
@@ -88,6 +89,13 @@ acl.defaultRole=
 ganglia.group=
 ganglia.port=8664
 
+# route to hive settings
+kylin.route.hive.enabled=false
+# default to the embedded server (jdbc:hive2://)
+#kylin.route.hive.url=
+#kylin.route.hive.username=
+#kylin.route.hive.password=
+
 ## Config for mail service
 
 # If true, will send email notification;
@@ -131,12 +139,3 @@ kylin.monitor.ext.log.base.dir = /tmp/kylin_log1,/tmp/kylin_log2
 #will create external hive table to query result csv file
 #will set to kylin_query_log by default if not config here
 kylin.monitor.query.log.parse.result.table = kylin_query_log
-
-##### Config for enabling query rerouting to Hive. Disabled by default. ######
-kylin.route.hive.enabled=false
-
-# If query rerouting is enabled, provide the hive configurations
-# Default value for kylin.route.hive.url will be pointing to the embedded server (jdbc:hive2://)
-kylin.route.hive.url=
-kylin.route.hive.username=
-kylin.route.hive.password=

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/143aa608/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index adeb5ba..eed7068 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -94,6 +94,11 @@
         </dependency>
 
         <dependency>
+            <groupId>net.sf.opencsv</groupId>
+            <artifactId>opencsv</artifactId>
+        </dependency>
+
+        <dependency>
             <groupId>org.springframework.boot</groupId>
             <artifactId>spring-boot-starter-web</artifactId>
             <version>0.5.0.M6</version>
@@ -360,10 +365,10 @@
                 </exclusion>
             </exclusions>
           </dependency>
-
         <dependency>
-            <groupId>net.sf.opencsv</groupId>
-            <artifactId>opencsv</artifactId>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-jdbc</artifactId>
+            <scope>provided</scope>
         </dependency>
 
         <!-- Tomcat Env -->

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/143aa608/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 81905c8..8a397cd 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -40,9 +40,6 @@ import java.util.Set;
 
 import javax.sql.DataSource;
 
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.calcite.sql.parser.impl.ParseException;
-import org.apache.calcite.sql.validate.SqlValidatorException;
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.client.Get;
@@ -65,7 +62,7 @@ import org.apache.kylin.rest.model.TableMeta;
 import org.apache.kylin.rest.request.PrepareSqlRequest;
 import org.apache.kylin.rest.request.SQLRequest;
 import org.apache.kylin.rest.response.SQLResponse;
-import org.apache.kylin.rest.util.HiveRerouteUtil;
+import org.apache.kylin.rest.util.HiveReroute;
 import org.apache.kylin.rest.util.QueryUtil;
 import org.apache.kylin.rest.util.Serializer;
 import org.slf4j.Logger;
@@ -102,7 +99,7 @@ public class QueryService extends BasicService {
         tableNameBase = cut < 0 ? DEFAULT_TABLE_PREFIX : metadataUrl.substring(0, cut);
         hbaseUrl = cut < 0 ? metadataUrl : metadataUrl.substring(cut + 1);
         userTableName = tableNameBase + USER_TABLE_NAME;
-        
+
         badQueryDetector.start();
     }
 
@@ -113,9 +110,9 @@ public class QueryService extends BasicService {
     public SQLResponse query(SQLRequest sqlRequest) throws Exception {
         try {
             badQueryDetector.queryStart(Thread.currentThread(), sqlRequest);
-            
+
             return queryWithSqlMassage(sqlRequest);
-            
+
         } finally {
             badQueryDetector.queryEnd(Thread.currentThread());
         }
@@ -373,33 +370,22 @@ public class QueryService extends BasicService {
                     oneRow.add((resultSet.getString(i + 1)));
                 }
 
-                results.add(new LinkedList<String>(oneRow));
+                results.add(new ArrayList<String>(oneRow));
                 oneRow.clear();
             }
         } catch (SQLException sqlException) {
-            // unsuccessful statement execution, retry with Hive on Spark. Code modification as part of the jira https://issues.apache.org/jira/browse/KYLIN-742
-	    boolean isExpectedCause = (ExceptionUtils.getRootCause(sqlException).getClass().equals(SqlValidatorException.class)) || (ExceptionUtils.getRootCause(sqlException).getClass().equals(ParseException.class));
-            KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-    	    if (isExpectedCause && kylinConfig.isHiveReroutingEnabled()) {
-        	logger.debug("query rerouting option enabled for Kylin");
-        	// running query to hive
-                HiveRerouteUtil rerouteUtil = new HiveRerouteUtil();
-                try {
-                    conn = rerouteUtil.createConnection(kylinConfig.getHiveRerouteUrl(), kylinConfig.getHiveRerouteUsername(), kylinConfig.getHiveReroutePassword());
-                    resultSet = rerouteUtil.executQuery(conn, sql);
-                    columnMetas = rerouteUtil.extractColumnMetadata(resultSet, columnMetas);
-                    results = rerouteUtil.extractResults(resultSet, results);
-                } catch (Exception exception) {
-                    logger.error("exception in re-routing the query to hive", exception);
-                    throw exception;
-                } finally {
-                    rerouteUtil.closeConnection(conn);
-                } 
-            } else {
-                logger.error("exception in running the query: " + sql);
-		throw sqlException;
-    	    }
-	    } finally {
+            // hive maybe able to answer where kylin failed
+            HiveReroute reroute = new HiveReroute();
+            if (reroute.shouldReroute(sqlException) == false) {
+                throw sqlException;
+            }
+            try {
+                reroute.query(sql, results, columnMetas);
+            } catch (Throwable ex) {
+                logger.error("Reroute hive failed", ex);
+                throw sqlException;
+            }
+        } finally {
             close(resultSet, stat, conn);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/143aa608/server/src/main/java/org/apache/kylin/rest/util/HiveReroute.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/util/HiveReroute.java b/server/src/main/java/org/apache/kylin/rest/util/HiveReroute.java
new file mode 100644
index 0000000..c83afb2
--- /dev/null
+++ b/server/src/main/java/org/apache/kylin/rest/util/HiveReroute.java
@@ -0,0 +1,142 @@
+/*
+ * 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.kylin.rest.util;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.calcite.sql.parser.impl.ParseException;
+import org.apache.calcite.sql.validate.SqlValidatorException;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.rest.model.SelectedColumnMeta;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HiveReroute {
+
+    private static final Logger logger = LoggerFactory.getLogger(HiveReroute.class);
+
+    public static final String HIVE_DRIVER_CLASS = "org.apache.hive.jdbc.HiveDriver";
+
+    public boolean shouldReroute(SQLException sqlException) {
+        Throwable rootCause = ExceptionUtils.getRootCause(sqlException);
+        boolean isExpectedCause = (rootCause instanceof SqlValidatorException || rootCause instanceof ParseException);
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        return isExpectedCause && kylinConfig.isHiveReroutingEnabled();
+    }
+
+    public void query(String sql, List<List<String>> results, List<SelectedColumnMeta> columnMetas) throws Exception {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        logger.debug("query rerouting option enabled for Kylin");
+
+        // running query to hive
+        Connection conn = null;
+        ResultSet resultSet = null;
+        try {
+            conn = createConnection(kylinConfig.getHiveRerouteUrl(), kylinConfig.getHiveRerouteUsername(), kylinConfig.getHiveReroutePassword());
+            resultSet = executQuery(conn, sql);
+            extractColumnMetadata(resultSet, columnMetas);
+            extractResults(resultSet, results);
+        } finally {
+            closeConnection(conn);
+        }
+    }
+
+    /**
+     * Create a connection to the Hive server by passing the required connection parameters.
+     * @param connectionURL: JDBC URL to connect to the Hive server
+     * @param username: Username to connect with (optional)
+     * @param password: Password to connect with (optional)
+     * @return: Connection object to the Hive server
+     * @throws Exception
+     */
+    private Connection createConnection(String connectionURL, String username, String password) throws Exception {
+        logger.info("rerouting to : " + connectionURL + " for executing the query");
+
+        Class.forName(HIVE_DRIVER_CLASS);
+        Connection connection = DriverManager.getConnection(connectionURL, username, password);
+        return connection;
+    }
+
+    /**
+     * Close the connection to the Hive server.
+     * @param connection: Connection object to be closed
+     */
+    private void closeConnection(Connection connection) {
+        if (null != connection) {
+            try {
+                connection.close();
+            } catch (SQLException sqlException) {
+                logger.error("failed to close connection", sqlException);
+            }
+        }
+    }
+
+    /**
+     * Execute a query in Hive.
+     * @param connection: Connection object to the Hive server
+     * @param query: Query to be executed
+     * @return: ResultSet object of the query executed
+     * @throws Exception
+     */
+    private ResultSet executQuery(Connection connection, String query) throws Exception {
+        Statement statement = null;
+        ResultSet resultSet = null;
+
+        statement = connection.createStatement();
+        resultSet = statement.executeQuery(query);
+        return resultSet;
+    }
+
+    private void extractColumnMetadata(ResultSet resultSet, List<SelectedColumnMeta> columnMetas) throws SQLException {
+        ResultSetMetaData metaData = null;
+        int columnCount = 0;
+
+        metaData = resultSet.getMetaData();
+        columnCount = metaData.getColumnCount();
+
+        // fill in selected column meta
+        for (int i = 1; i <= columnCount; ++i) {
+            columnMetas.add(new SelectedColumnMeta(metaData.isAutoIncrement(i), metaData.isCaseSensitive(i), false, metaData.isCurrency(i), metaData.isNullable(i), false, metaData.getColumnDisplaySize(i), metaData.getColumnLabel(i), metaData.getColumnName(i), null, null, null, metaData.getPrecision(i), metaData.getScale(i), metaData.getColumnType(i), metaData.getColumnTypeName(i), metaData.isReadOnly(i), false, false));
+        }
+    }
+
+    private void extractResults(ResultSet resultSet, List<List<String>> results) throws SQLException {
+        List<String> oneRow = new LinkedList<String>();
+
+        while (resultSet.next()) {
+            //logger.debug("resultSet value: " + resultSet.getString(1));
+            for (int i = 0; i < resultSet.getMetaData().getColumnCount(); i++) {
+                oneRow.add((resultSet.getString(i + 1)));
+            }
+
+            results.add(new ArrayList<String>(oneRow));
+            oneRow.clear();
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/143aa608/server/src/main/java/org/apache/kylin/rest/util/HiveRerouteUtil.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/util/HiveRerouteUtil.java b/server/src/main/java/org/apache/kylin/rest/util/HiveRerouteUtil.java
deleted file mode 100644
index ca1e048..0000000
--- a/server/src/main/java/org/apache/kylin/rest/util/HiveRerouteUtil.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.rest.util;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.kylin.rest.model.SelectedColumnMeta;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** @author murkrishn **/
-public class HiveRerouteUtil {
-
-    private static final Logger logger = LoggerFactory.getLogger(HiveRerouteUtil.class);
-    public static final String driverName = "org.apache.hive.jdbc.HiveDriver";
-
-    /**
-     * Create a connection to the Hive server by passing the required connection parameters.
-     * @param connectionURL: JDBC URL to connect to the Hive server
-     * @param username: Username to connect with (optional)
-     * @param password: Password to connect with (optional)
-     * @return: Connection object to the Hive server
-     * @throws Exception
-     */
-    public Connection createConnection (String connectionURL, String username, String password) throws Exception {
-        logger.info("rerouting to : " + connectionURL + " for executing the query");
-        
-        try {
-            Class.forName(driverName);
-        } catch (ClassNotFoundException classNotFoundException) {
-            throw classNotFoundException;
-        }
-        
-        Connection connection = DriverManager.getConnection(connectionURL, username, password);
-        return connection;
-    }
-    
-    /**
-     * Close the connection to the Hive server.
-     * @param connection: Connection object to be closed
-     */
-    public void closeConnection(Connection connection) {
-        if (null != connection) {
-            try {
-                connection.close();
-            } catch (SQLException sqlException) {
-                logger.error("failed to close connection", sqlException);
-            }
-        }
-    }
-    
-    /**
-     * Execute a query in Hive.
-     * @param connection: Connection object to the Hive server
-     * @param query: Query to be executed
-     * @return: ResultSet object of the query executed
-     * @throws Exception
-     */
-    public ResultSet executQuery (Connection connection, String query) throws Exception {
-        Statement statement = null;
-        ResultSet resultSet = null;
-        
-        try {
-            statement = connection.createStatement();
-            resultSet = statement.executeQuery(query);
-            return resultSet;
-        } catch (SQLException sqlException) {
-            throw sqlException;
-        }
-    }
-    
-    public List<SelectedColumnMeta> extractColumnMetadata (ResultSet resultSet, List<SelectedColumnMeta> columnMetas) throws SQLException {
-        ResultSetMetaData metaData = null;
-        int columnCount = 0;
-        
-        try {
-            metaData = resultSet.getMetaData();
-            columnCount = metaData.getColumnCount();
-            
-            // fill in selected column meta
-            for (int i = 1; i <= columnCount; ++i) {
-                columnMetas.add(new SelectedColumnMeta(metaData.isAutoIncrement(i), metaData.isCaseSensitive(i), false, metaData.isCurrency(i), metaData.isNullable(i), false, metaData.getColumnDisplaySize(i), metaData.getColumnLabel(i), metaData.getColumnName(i), null, null, null, metaData.getPrecision(i), metaData.getScale(i), metaData.getColumnType(i), metaData.getColumnTypeName(i), metaData.isReadOnly(i), false, false));
-            }
-            
-            return columnMetas;
-        } catch (SQLException sqlException) {
-            throw sqlException;
-        }
-    }
-    
-    public List<List<String>> extractResults (ResultSet resultSet, List<List<String>> results) throws SQLException {
-        List<String> oneRow = new LinkedList<String>();
-        
-        try {
-            while (resultSet.next()) {
-                //logger.debug("resultSet value: " + resultSet.getString(1));
-                for (int i = 0; i < resultSet.getMetaData().getColumnCount(); i++) {
-                    oneRow.add((resultSet.getString(i + 1)));
-                }
-
-                results.add(new LinkedList<String>(oneRow));
-                oneRow.clear();
-            }
-            
-            return results;
-        } catch (SQLException sqlException) {
-            throw sqlException;
-        }
-    }
-}
\ No newline at end of file


[42/45] incubator-kylin git commit: KYLIN-958 allow to save non-structure change in DataModelDesc

Posted by sh...@apache.org.
KYLIN-958 allow to save non-structure change in DataModelDesc

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

Branch: refs/heads/master
Commit: b545f00ababb6a20623f7c40e7966f75256db9d0
Parents: a64759c
Author: shaofengshi <sh...@apache.org>
Authored: Thu Oct 8 15:59:38 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Oct 8 15:59:38 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/rest/controller/CubeController.java     | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b545f00a/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java b/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
index 3d0fca3..8193f5e 100644
--- a/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
+++ b/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
@@ -349,9 +349,10 @@ public class CubeController extends BasicController {
         }
 
         MetadataManager metadataManager = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
-        // KYLIN-958: disallow data model change
+        // KYLIN-958: disallow data model structure change
+        DataModelDesc modelDesc = null;
         if (StringUtils.isNotEmpty(cubeRequest.getModelDescData())) {
-            DataModelDesc modelDesc = deserializeDataModelDesc(cubeRequest);
+            modelDesc = deserializeDataModelDesc(cubeRequest);
             if (modelDesc == null) {
                 return cubeRequest;
             }
@@ -380,6 +381,9 @@ public class CubeController extends BasicController {
         }
 
         try {
+            if (modelDesc != null)
+                metadataManager.updateDataModelDesc(modelDesc);
+            
             CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
             String projectName = (null == cubeRequest.getProject()) ? ProjectInstance.DEFAULT_PROJECT_NAME : cubeRequest.getProject();
             desc = cubeService.updateCubeAndDesc(cube, desc, projectName);


[06/45] incubator-kylin git commit: KYLIN-1039 Add test case to confirm the issue exists or not

Posted by sh...@apache.org.
KYLIN-1039 Add test case to confirm the issue exists or not


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

Branch: refs/heads/master
Commit: 038eb5913f78a3bf38977a0a013f4f138b95817a
Parents: d86829a
Author: Li, Yang <ya...@ebay.com>
Authored: Tue Sep 22 14:52:58 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Tue Sep 22 14:52:58 2015 +0800

----------------------------------------------------------------------
 query/src/test/resources/query/sql/query81.sql  | 23 ++++++++++++++++++++
 .../coprocessor/observer/ObserverEnabler.java   |  8 +++++--
 2 files changed, 29 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/038eb591/query/src/test/resources/query/sql/query81.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query81.sql b/query/src/test/resources/query/sql/query81.sql
new file mode 100644
index 0000000..7302a7d
--- /dev/null
+++ b/query/src/test/resources/query/sql/query81.sql
@@ -0,0 +1,23 @@
+--
+-- 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.
+--
+
+select test_cal_dt.week_beg_dt, sum(price) as GMV
+ from test_kylin_fact 
+ inner JOIN edw.test_cal_dt as test_cal_dt  ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+ where test_cal_dt.week_beg_dt between DATE '2013-09-01' and DATE '2013-10-01' and (lstg_format_name='FP-GTC' or 'a' = 'b')
+ group by test_cal_dt.week_beg_dt
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/038eb591/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
index 7cfc23c..f0f7ed5 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
@@ -105,12 +105,16 @@ public class ObserverEnabler {
 
         String forceFlag = System.getProperty(FORCE_COPROCESSOR);
         if (forceFlag != null) {
-            return Boolean.parseBoolean(forceFlag);
+            boolean r = Boolean.parseBoolean(forceFlag);
+            logger.info("Coprocessor is " + (r ? "enabled" : "disabled") + " according to sys prop " + FORCE_COPROCESSOR);
+            return r;
         }
 
         Boolean cubeOverride = CUBE_OVERRIDES.get(cube.getName());
         if (cubeOverride != null) {
-            return cubeOverride.booleanValue();
+            boolean r = cubeOverride.booleanValue();
+            logger.info("Coprocessor is " + (r ? "enabled" : "disabled") + " according to cube overrides");
+            return r;
         }
 
         if (RowValueDecoder.hasMemHungryCountDistinct(rowValueDecoders)) {


[23/45] incubator-kylin git commit: KYLIN-1047 Upgrade to Calcite 1.4 (with Yerui Sun)

Posted by sh...@apache.org.
KYLIN-1047 Upgrade to Calcite 1.4 (with Yerui Sun)


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

Branch: refs/heads/master
Commit: 80fbd77e7016616fe551a450d85e6f6b026a209a
Parents: 143aa60
Author: Yang Li <li...@apache.org>
Authored: Sat Sep 26 20:15:23 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Wed Sep 30 14:55:05 2015 +0800

----------------------------------------------------------------------
 .../adapter/enumerable/EnumerableJoin.java      |  146 --
 .../apache/calcite/runtime/SqlFunctions.java    | 1315 ------------------
 .../calcite/sql2rel/SqlToRelConverter.java      |  253 +---
 .../java/org/apache/kylin/jdbc/KylinMeta.java   |    9 +-
 pom.xml                                         |    2 +-
 .../apache/kylin/query/relnode/OLAPJoinRel.java |   24 +-
 .../kylin/query/relnode/OLAPTableScan.java      |   13 +-
 7 files changed, 100 insertions(+), 1662 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/80fbd77e/atopcalcite/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java b/atopcalcite/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
deleted file mode 100644
index 17f1b38..0000000
--- a/atopcalcite/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.adapter.enumerable;
-
-import java.util.Set;
-
-import org.apache.calcite.linq4j.tree.BlockBuilder;
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.linq4j.tree.Expressions;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.InvalidRelException;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelNodes;
-import org.apache.calcite.rel.core.EquiJoin;
-import org.apache.calcite.rel.core.JoinInfo;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.util.BuiltInMethod;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.Util;
-
-import com.google.common.collect.ImmutableList;
-
-/*
- * OVERRIDE POINT:
- * - constructor was private instead of protected
- */
-
-/** Implementation of {@link org.apache.calcite.rel.core.Join} in
- * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
-public class EnumerableJoin extends EquiJoin implements EnumerableRel {
-    /** Creates an EnumerableJoin.
-     *
-     * <p>Use {@link #create} unless you know what you're doing. */
-    public EnumerableJoin(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition, ImmutableIntList leftKeys, ImmutableIntList rightKeys, JoinRelType joinType, Set<String> variablesStopped) throws InvalidRelException {
-        super(cluster, traits, left, right, condition, leftKeys, rightKeys, joinType, variablesStopped);
-    }
-
-    /** Creates an EnumerableJoin. */
-    public static EnumerableJoin create(RelNode left, RelNode right, RexNode condition, ImmutableIntList leftKeys, ImmutableIntList rightKeys, JoinRelType joinType, Set<String> variablesStopped) throws InvalidRelException {
-        final RelOptCluster cluster = left.getCluster();
-        final RelTraitSet traitSet = cluster.traitSetOf(EnumerableConvention.INSTANCE);
-        return new EnumerableJoin(cluster, traitSet, left, right, condition, leftKeys, rightKeys, joinType, variablesStopped);
-    }
-
-    @Override
-    public EnumerableJoin copy(RelTraitSet traitSet, RexNode condition, RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
-        final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
-        assert joinInfo.isEqui();
-        try {
-            return new EnumerableJoin(getCluster(), traitSet, left, right, condition, joinInfo.leftKeys, joinInfo.rightKeys, joinType, variablesStopped);
-        } catch (InvalidRelException e) {
-            // Semantic error not possible. Must be a bug. Convert to
-            // internal error.
-            throw new AssertionError(e);
-        }
-    }
-
-    @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-        double rowCount = RelMetadataQuery.getRowCount(this);
-
-        // Joins can be flipped, and for many algorithms, both versions are viable
-        // and have the same cost. To make the results stable between versions of
-        // the planner, make one of the versions slightly more expensive.
-        switch (joinType) {
-        case RIGHT:
-            rowCount = addEpsilon(rowCount);
-            break;
-        default:
-            if (RelNodes.COMPARATOR.compare(left, right) > 0) {
-                rowCount = addEpsilon(rowCount);
-            }
-        }
-
-        // Cheaper if the smaller number of rows is coming from the LHS.
-        // Model this by adding L log L to the cost.
-        final double rightRowCount = right.getRows();
-        final double leftRowCount = left.getRows();
-        if (Double.isInfinite(leftRowCount)) {
-            rowCount = leftRowCount;
-        } else {
-            rowCount += Util.nLogN(leftRowCount);
-        }
-        if (Double.isInfinite(rightRowCount)) {
-            rowCount = rightRowCount;
-        } else {
-            rowCount += rightRowCount;
-        }
-        return planner.getCostFactory().makeCost(rowCount, 0, 0);
-    }
-
-    private double addEpsilon(double d) {
-        assert d >= 0d;
-        final double d0 = d;
-        if (d < 10) {
-            // For small d, adding 1 would change the value significantly.
-            d *= 1.001d;
-            if (d != d0) {
-                return d;
-            }
-        }
-        // For medium d, add 1. Keeps integral values integral.
-        ++d;
-        if (d != d0) {
-            return d;
-        }
-        // For large d, adding 1 might not change the value. Add .1%.
-        // If d is NaN, this still will probably not change the value. That's OK.
-        d *= 1.001d;
-        return d;
-    }
-
-    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-        BlockBuilder builder = new BlockBuilder();
-        final Result leftResult = implementor.visitChild(this, 0, (EnumerableRel) left, pref);
-        Expression leftExpression = builder.append("left", leftResult.block);
-        final Result rightResult = implementor.visitChild(this, 1, (EnumerableRel) right, pref);
-        Expression rightExpression = builder.append("right", rightResult.block);
-        final PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), getRowType(), pref.preferArray());
-        final PhysType keyPhysType = leftResult.physType.project(leftKeys, JavaRowFormat.LIST);
-        return implementor.result(physType, builder.append(Expressions.call(leftExpression, BuiltInMethod.JOIN.method, Expressions.list(rightExpression, leftResult.physType.generateAccessor(leftKeys), rightResult.physType.generateAccessor(rightKeys), EnumUtils.joinSelector(joinType, physType, ImmutableList.of(leftResult.physType, rightResult.physType))).append(Util.first(keyPhysType.comparer(), Expressions.constant(null))).append(Expressions.constant(joinType.generatesNullsOnLeft())).append(Expressions.constant(joinType.generatesNullsOnRight())))).toBlock());
-    }
-
-}
-
-// End EnumerableJoin.java

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/80fbd77e/atopcalcite/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/atopcalcite/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
deleted file mode 100644
index 4b3d589..0000000
--- a/atopcalcite/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ /dev/null
@@ -1,1315 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.runtime;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.math.MathContext;
-import java.sql.SQLException;
-import java.sql.Timestamp;
-import java.text.DecimalFormat;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TimeZone;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.regex.Pattern;
-
-import org.apache.calcite.DataContext;
-import org.apache.calcite.avatica.util.ByteString;
-import org.apache.calcite.avatica.util.DateTimeUtils;
-import org.apache.calcite.linq4j.Enumerable;
-import org.apache.calcite.linq4j.Linq4j;
-import org.apache.calcite.linq4j.function.Deterministic;
-import org.apache.calcite.linq4j.function.Function1;
-import org.apache.calcite.linq4j.function.NonDeterministic;
-import org.apache.calcite.linq4j.tree.Primitive;
-
-/**
- * OVERRIDE POINT:
- * - divide(BigDecimal,BigDecimal), was `b0.divide(b1)`, now `b0.divide(b1, MathContext.DECIMAL64);`
- * Helper methods to implement SQL functions in generated code.
- *
- * <p>Not present: and, or, not (builtin operators are better, because they
- * use lazy evaluation. Implementations do not check for null values; the
- * calling code must do that.</p>
- *
- * <p>Many of the functions do not check for null values. This is intentional.
- * If null arguments are possible, the code-generation framework checks for
- * nulls before calling the functions.</p>
- */
-@SuppressWarnings("UnnecessaryUnboxing")
-@Deterministic
-public class SqlFunctions {
-    private static final DecimalFormat DOUBLE_FORMAT = new DecimalFormat("0.0E0");
-
-    private static final TimeZone LOCAL_TZ = TimeZone.getDefault();
-
-    private static final Function1<List<Object>, Enumerable<Object>> LIST_AS_ENUMERABLE = new Function1<List<Object>, Enumerable<Object>>() {
-        public Enumerable<Object> apply(List<Object> list) {
-            return Linq4j.asEnumerable(list);
-        }
-    };
-
-    /** Holds, for each thread, a map from sequence name to sequence current
-     * value.
-     *
-     * <p>This is a straw man of an implementation whose main goal is to prove
-     * that sequences can be parsed, validated and planned. A real application
-     * will want persistent values for sequences, shared among threads. */
-    private static final ThreadLocal<Map<String, AtomicLong>> THREAD_SEQUENCES = new ThreadLocal<Map<String, AtomicLong>>() {
-        @Override
-        protected Map<String, AtomicLong> initialValue() {
-            return new HashMap<String, AtomicLong>();
-        }
-    };
-
-    private SqlFunctions() {
-    }
-
-    /** SQL SUBSTRING(string FROM ... FOR ...) function. */
-    public static String substring(String s, int from, int for_) {
-        return s.substring(from - 1, Math.min(from - 1 + for_, s.length()));
-    }
-
-    /** SQL SUBSTRING(string FROM ...) function. */
-    public static String substring(String s, int from) {
-        return s.substring(from - 1);
-    }
-
-    /** SQL UPPER(string) function. */
-    public static String upper(String s) {
-        return s.toUpperCase();
-    }
-
-    /** SQL LOWER(string) function. */
-    public static String lower(String s) {
-        return s.toLowerCase();
-    }
-
-    /** SQL INITCAP(string) function. */
-    public static String initcap(String s) {
-        // Assumes Alpha as [A-Za-z0-9]
-        // white space is treated as everything else.
-        final int len = s.length();
-        boolean start = true;
-        final StringBuilder newS = new StringBuilder();
-
-        for (int i = 0; i < len; i++) {
-            char curCh = s.charAt(i);
-            final int c = (int) curCh;
-            if (start) { // curCh is whitespace or first character of word.
-                if (c > 47 && c < 58) { // 0-9
-                    start = false;
-                } else if (c > 64 && c < 91) { // A-Z
-                    start = false;
-                } else if (c > 96 && c < 123) { // a-z
-                    start = false;
-                    curCh = (char) (c - 32); // Uppercase this character
-                }
-                // else {} whitespace
-            } else { // Inside of a word or white space after end of word.
-                if (c > 47 && c < 58) { // 0-9
-                    // noop
-                } else if (c > 64 && c < 91) { // A-Z
-                    curCh = (char) (c + 32); // Lowercase this character
-                } else if (c > 96 && c < 123) { // a-z
-                    // noop
-                } else { // whitespace
-                    start = true;
-                }
-            }
-            newS.append(curCh);
-        } // for each character in s
-        return newS.toString();
-    }
-
-    /** SQL CHARACTER_LENGTH(string) function. */
-    public static int charLength(String s) {
-        return s.length();
-    }
-
-    /** SQL {@code string || string} operator. */
-    public static String concat(String s0, String s1) {
-        return s0 + s1;
-    }
-
-    /** SQL {@code binary || binary} operator. */
-    public static ByteString concat(ByteString s0, ByteString s1) {
-        return s0.concat(s1);
-    }
-
-    /** SQL {@code RTRIM} function applied to string. */
-    public static String rtrim(String s) {
-        return trim_(s, false, true, ' ');
-    }
-
-    /** SQL {@code LTRIM} function. */
-    public static String ltrim(String s) {
-        return trim_(s, true, false, ' ');
-    }
-
-    /** SQL {@code TRIM(... seek FROM s)} function. */
-    public static String trim(boolean leading, boolean trailing, String seek, String s) {
-        return trim_(s, leading, trailing, seek.charAt(0));
-    }
-
-    /** SQL {@code TRIM} function. */
-    private static String trim_(String s, boolean left, boolean right, char c) {
-        int j = s.length();
-        if (right) {
-            for (;;) {
-                if (j == 0) {
-                    return "";
-                }
-                if (s.charAt(j - 1) != c) {
-                    break;
-                }
-                --j;
-            }
-        }
-        int i = 0;
-        if (left) {
-            for (;;) {
-                if (i == j) {
-                    return "";
-                }
-                if (s.charAt(i) != c) {
-                    break;
-                }
-                ++i;
-            }
-        }
-        return s.substring(i, j);
-    }
-
-    /** SQL {@code TRIM} function applied to binary string. */
-    public static ByteString trim(ByteString s) {
-        return trim_(s, true, true);
-    }
-
-    /** Helper for CAST. */
-    public static ByteString rtrim(ByteString s) {
-        return trim_(s, false, true);
-    }
-
-    /** SQL {@code TRIM} function applied to binary string. */
-    private static ByteString trim_(ByteString s, boolean left, boolean right) {
-        int j = s.length();
-        if (right) {
-            for (;;) {
-                if (j == 0) {
-                    return ByteString.EMPTY;
-                }
-                if (s.byteAt(j - 1) != 0) {
-                    break;
-                }
-                --j;
-            }
-        }
-        int i = 0;
-        if (left) {
-            for (;;) {
-                if (i == j) {
-                    return ByteString.EMPTY;
-                }
-                if (s.byteAt(i) != 0) {
-                    break;
-                }
-                ++i;
-            }
-        }
-        return s.substring(i, j);
-    }
-
-    /** SQL {@code OVERLAY} function. */
-    public static String overlay(String s, String r, int start) {
-        if (s == null || r == null) {
-            return null;
-        }
-        return s.substring(0, start - 1) + r + s.substring(start - 1 + r.length());
-    }
-
-    /** SQL {@code OVERLAY} function. */
-    public static String overlay(String s, String r, int start, int length) {
-        if (s == null || r == null) {
-            return null;
-        }
-        return s.substring(0, start - 1) + r + s.substring(start - 1 + length);
-    }
-
-    /** SQL {@code OVERLAY} function applied to binary strings. */
-    public static ByteString overlay(ByteString s, ByteString r, int start) {
-        if (s == null || r == null) {
-            return null;
-        }
-        return s.substring(0, start - 1).concat(r).concat(s.substring(start - 1 + r.length()));
-    }
-
-    /** SQL {@code OVERLAY} function applied to binary strings. */
-    public static ByteString overlay(ByteString s, ByteString r, int start, int length) {
-        if (s == null || r == null) {
-            return null;
-        }
-        return s.substring(0, start - 1).concat(r).concat(s.substring(start - 1 + length));
-    }
-
-    /** SQL {@code LIKE} function. */
-    public static boolean like(String s, String pattern) {
-        final String regex = Like.sqlToRegexLike(pattern, null);
-        return Pattern.matches(regex, s);
-    }
-
-    /** SQL {@code LIKE} function with escape. */
-    public static boolean like(String s, String pattern, String escape) {
-        final String regex = Like.sqlToRegexLike(pattern, escape);
-        return Pattern.matches(regex, s);
-    }
-
-    /** SQL {@code SIMILAR} function. */
-    public static boolean similar(String s, String pattern) {
-        final String regex = Like.sqlToRegexSimilar(pattern, null);
-        return Pattern.matches(regex, s);
-    }
-
-    /** SQL {@code SIMILAR} function with escape. */
-    public static boolean similar(String s, String pattern, String escape) {
-        final String regex = Like.sqlToRegexSimilar(pattern, escape);
-        return Pattern.matches(regex, s);
-    }
-
-    // =
-
-    /** SQL = operator applied to Object values (including String; neither
-     * side may be null). */
-    public static boolean eq(Object b0, Object b1) {
-        return b0.equals(b1);
-    }
-
-    /** SQL = operator applied to BigDecimal values (neither may be null). */
-    public static boolean eq(BigDecimal b0, BigDecimal b1) {
-        return b0.stripTrailingZeros().equals(b1.stripTrailingZeros());
-    }
-
-    // <>
-
-    /** SQL &lt;&gt; operator applied to Object values (including String;
-     * neither side may be null). */
-    public static boolean ne(Object b0, Object b1) {
-        return !b0.equals(b1);
-    }
-
-    /** SQL &lt;&gt; operator applied to BigDecimal values. */
-    public static boolean ne(BigDecimal b0, BigDecimal b1) {
-        return b0.compareTo(b1) != 0;
-    }
-
-    // <
-
-    /** SQL &lt; operator applied to boolean values. */
-    public static boolean lt(boolean b0, boolean b1) {
-        return compare(b0, b1) < 0;
-    }
-
-    /** SQL &lt; operator applied to String values. */
-    public static boolean lt(String b0, String b1) {
-        return b0.compareTo(b1) < 0;
-    }
-
-    /** SQL &lt; operator applied to ByteString values. */
-    public static boolean lt(ByteString b0, ByteString b1) {
-        return b0.compareTo(b1) < 0;
-    }
-
-    /** SQL &lt; operator applied to BigDecimal values. */
-    public static boolean lt(BigDecimal b0, BigDecimal b1) {
-        return b0.compareTo(b1) < 0;
-    }
-
-    // <=
-
-    /** SQL &le; operator applied to boolean values. */
-    public static boolean le(boolean b0, boolean b1) {
-        return compare(b0, b1) <= 0;
-    }
-
-    /** SQL &le; operator applied to String values. */
-    public static boolean le(String b0, String b1) {
-        return b0.compareTo(b1) <= 0;
-    }
-
-    /** SQL &le; operator applied to ByteString values. */
-    public static boolean le(ByteString b0, ByteString b1) {
-        return b0.compareTo(b1) <= 0;
-    }
-
-    /** SQL &le; operator applied to BigDecimal values. */
-    public static boolean le(BigDecimal b0, BigDecimal b1) {
-        return b0.compareTo(b1) <= 0;
-    }
-
-    // >
-
-    /** SQL &gt; operator applied to boolean values. */
-    public static boolean gt(boolean b0, boolean b1) {
-        return compare(b0, b1) > 0;
-    }
-
-    /** SQL &gt; operator applied to String values. */
-    public static boolean gt(String b0, String b1) {
-        return b0.compareTo(b1) > 0;
-    }
-
-    /** SQL &gt; operator applied to ByteString values. */
-    public static boolean gt(ByteString b0, ByteString b1) {
-        return b0.compareTo(b1) > 0;
-    }
-
-    /** SQL &gt; operator applied to BigDecimal values. */
-    public static boolean gt(BigDecimal b0, BigDecimal b1) {
-        return b0.compareTo(b1) > 0;
-    }
-
-    // >=
-
-    /** SQL &ge; operator applied to boolean values. */
-    public static boolean ge(boolean b0, boolean b1) {
-        return compare(b0, b1) >= 0;
-    }
-
-    /** SQL &ge; operator applied to String values. */
-    public static boolean ge(String b0, String b1) {
-        return b0.compareTo(b1) >= 0;
-    }
-
-    /** SQL &ge; operator applied to ByteString values. */
-    public static boolean ge(ByteString b0, ByteString b1) {
-        return b0.compareTo(b1) >= 0;
-    }
-
-    /** SQL &ge; operator applied to BigDecimal values. */
-    public static boolean ge(BigDecimal b0, BigDecimal b1) {
-        return b0.compareTo(b1) >= 0;
-    }
-
-    // +
-
-    /** SQL <code>+</code> operator applied to int values. */
-    public static int plus(int b0, int b1) {
-        return b0 + b1;
-    }
-
-    /** SQL <code>+</code> operator applied to int values; left side may be
-     * null. */
-    public static Integer plus(Integer b0, int b1) {
-        return b0 == null ? null : (b0 + b1);
-    }
-
-    /** SQL <code>+</code> operator applied to int values; right side may be
-     * null. */
-    public static Integer plus(int b0, Integer b1) {
-        return b1 == null ? null : (b0 + b1);
-    }
-
-    /** SQL <code>+</code> operator applied to nullable int values. */
-    public static Integer plus(Integer b0, Integer b1) {
-        return (b0 == null || b1 == null) ? null : (b0 + b1);
-    }
-
-    /** SQL <code>+</code> operator applied to nullable long and int values. */
-    public static Long plus(Long b0, Integer b1) {
-        return (b0 == null || b1 == null) ? null : (b0.longValue() + b1.longValue());
-    }
-
-    /** SQL <code>+</code> operator applied to nullable int and long values. */
-    public static Long plus(Integer b0, Long b1) {
-        return (b0 == null || b1 == null) ? null : (b0.longValue() + b1.longValue());
-    }
-
-    /** SQL <code>+</code> operator applied to BigDecimal values. */
-    public static BigDecimal plus(BigDecimal b0, BigDecimal b1) {
-        return (b0 == null || b1 == null) ? null : b0.add(b1);
-    }
-
-    // -
-
-    /** SQL <code>-</code> operator applied to int values. */
-    public static int minus(int b0, int b1) {
-        return b0 - b1;
-    }
-
-    /** SQL <code>-</code> operator applied to int values; left side may be
-     * null. */
-    public static Integer minus(Integer b0, int b1) {
-        return b0 == null ? null : (b0 - b1);
-    }
-
-    /** SQL <code>-</code> operator applied to int values; right side may be
-     * null. */
-    public static Integer minus(int b0, Integer b1) {
-        return b1 == null ? null : (b0 - b1);
-    }
-
-    /** SQL <code>-</code> operator applied to nullable int values. */
-    public static Integer minus(Integer b0, Integer b1) {
-        return (b0 == null || b1 == null) ? null : (b0 - b1);
-    }
-
-    /** SQL <code>-</code> operator applied to nullable long and int values. */
-    public static Long minus(Long b0, Integer b1) {
-        return (b0 == null || b1 == null) ? null : (b0.longValue() - b1.longValue());
-    }
-
-    /** SQL <code>-</code> operator applied to nullable int and long values. */
-    public static Long minus(Integer b0, Long b1) {
-        return (b0 == null || b1 == null) ? null : (b0.longValue() - b1.longValue());
-    }
-
-    /** SQL <code>-</code> operator applied to BigDecimal values. */
-    public static BigDecimal minus(BigDecimal b0, BigDecimal b1) {
-        return (b0 == null || b1 == null) ? null : b0.subtract(b1);
-    }
-
-    // /
-
-    /** SQL <code>/</code> operator applied to int values. */
-    public static int divide(int b0, int b1) {
-        return b0 / b1;
-    }
-
-    /** SQL <code>/</code> operator applied to int values; left side may be
-     * null. */
-    public static Integer divide(Integer b0, int b1) {
-        return b0 == null ? null : (b0 / b1);
-    }
-
-    /** SQL <code>/</code> operator applied to int values; right side may be
-     * null. */
-    public static Integer divide(int b0, Integer b1) {
-        return b1 == null ? null : (b0 / b1);
-    }
-
-    /** SQL <code>/</code> operator applied to nullable int values. */
-    public static Integer divide(Integer b0, Integer b1) {
-        return (b0 == null || b1 == null) ? null : (b0 / b1);
-    }
-
-    /** SQL <code>/</code> operator applied to nullable long and int values. */
-    public static Long divide(Long b0, Integer b1) {
-        return (b0 == null || b1 == null) ? null : (b0.longValue() / b1.longValue());
-    }
-
-    /** SQL <code>/</code> operator applied to nullable int and long values. */
-    public static Long divide(Integer b0, Long b1) {
-        return (b0 == null || b1 == null) ? null : (b0.longValue() / b1.longValue());
-    }
-
-    /** SQL <code>/</code> operator applied to BigDecimal values. */
-    public static BigDecimal divide(BigDecimal b0, BigDecimal b1) {
-        return (b0 == null || b1 == null) ? null : b0.divide(b1, MathContext.DECIMAL64);
-    }
-
-    // *
-
-    /** SQL <code>*</code> operator applied to int values. */
-    public static int multiply(int b0, int b1) {
-        return b0 * b1;
-    }
-
-    /** SQL <code>*</code> operator applied to int values; left side may be
-     * null. */
-    public static Integer multiply(Integer b0, int b1) {
-        return b0 == null ? null : (b0 * b1);
-    }
-
-    /** SQL <code>*</code> operator applied to int values; right side may be
-     * null. */
-    public static Integer multiply(int b0, Integer b1) {
-        return b1 == null ? null : (b0 * b1);
-    }
-
-    /** SQL <code>*</code> operator applied to nullable int values. */
-    public static Integer multiply(Integer b0, Integer b1) {
-        return (b0 == null || b1 == null) ? null : (b0 * b1);
-    }
-
-    /** SQL <code>*</code> operator applied to nullable long and int values. */
-    public static Long multiply(Long b0, Integer b1) {
-        return (b0 == null || b1 == null) ? null : (b0.longValue() * b1.longValue());
-    }
-
-    /** SQL <code>*</code> operator applied to nullable int and long values. */
-    public static Long multiply(Integer b0, Long b1) {
-        return (b0 == null || b1 == null) ? null : (b0.longValue() * b1.longValue());
-    }
-
-    /** SQL <code>*</code> operator applied to BigDecimal values. */
-    public static BigDecimal multiply(BigDecimal b0, BigDecimal b1) {
-        return (b0 == null || b1 == null) ? null : b0.multiply(b1);
-    }
-
-    // EXP
-
-    /** SQL <code>EXP</code> operator applied to double values. */
-    public static double exp(double b0) {
-        return Math.exp(b0);
-    }
-
-    public static double exp(long b0) {
-        return Math.exp(b0);
-    }
-
-    // POWER
-
-    /** SQL <code>POWER</code> operator applied to double values. */
-    public static double power(double b0, double b1) {
-        return Math.pow(b0, b1);
-    }
-
-    public static double power(long b0, long b1) {
-        return Math.pow(b0, b1);
-    }
-
-    public static double power(long b0, BigDecimal b1) {
-        return Math.pow(b0, b1.doubleValue());
-    }
-
-    // LN
-
-    /** SQL {@code LN(number)} function applied to double values. */
-    public static double ln(double d) {
-        return Math.log(d);
-    }
-
-    /** SQL {@code LN(number)} function applied to long values. */
-    public static double ln(long b0) {
-        return Math.log(b0);
-    }
-
-    /** SQL {@code LN(number)} function applied to BigDecimal values. */
-    public static double ln(BigDecimal d) {
-        return Math.log(d.doubleValue());
-    }
-
-    // LOG10
-
-    /** SQL <code>LOG10(numeric)</code> operator applied to double values. */
-    public static double log10(double b0) {
-        return Math.log10(b0);
-    }
-
-    /** SQL {@code LOG10(number)} function applied to long values. */
-    public static double log10(long b0) {
-        return Math.log10(b0);
-    }
-
-    /** SQL {@code LOG10(number)} function applied to BigDecimal values. */
-    public static double log10(BigDecimal d) {
-        return Math.log10(d.doubleValue());
-    }
-
-    // MOD
-
-    /** SQL <code>MOD</code> operator applied to byte values. */
-    public static byte mod(byte b0, byte b1) {
-        return (byte) (b0 % b1);
-    }
-
-    /** SQL <code>MOD</code> operator applied to short values. */
-    public static short mod(short b0, short b1) {
-        return (short) (b0 % b1);
-    }
-
-    /** SQL <code>MOD</code> operator applied to int values. */
-    public static int mod(int b0, int b1) {
-        return b0 % b1;
-    }
-
-    /** SQL <code>MOD</code> operator applied to long values. */
-    public static long mod(long b0, long b1) {
-        return b0 % b1;
-    }
-
-    // temporary
-    public static BigDecimal mod(BigDecimal b0, int b1) {
-        return mod(b0, BigDecimal.valueOf(b1));
-    }
-
-    // temporary
-    public static int mod(int b0, BigDecimal b1) {
-        return mod(b0, b1.intValue());
-    }
-
-    public static BigDecimal mod(BigDecimal b0, BigDecimal b1) {
-        final BigDecimal[] bigDecimals = b0.divideAndRemainder(b1);
-        return bigDecimals[1];
-    }
-
-    // FLOOR
-
-    public static double floor(double b0) {
-        return Math.floor(b0);
-    }
-
-    public static float floor(float b0) {
-        return (float) Math.floor(b0);
-    }
-
-    public static BigDecimal floor(BigDecimal b0) {
-        return b0.setScale(0, BigDecimal.ROUND_FLOOR);
-    }
-
-    /** SQL <code>FLOOR</code> operator applied to byte values. */
-    public static byte floor(byte b0, byte b1) {
-        return (byte) floor((int) b0, (int) b1);
-    }
-
-    /** SQL <code>FLOOR</code> operator applied to short values. */
-    public static short floor(short b0, short b1) {
-        return (short) floor((int) b0, (int) b1);
-    }
-
-    /** SQL <code>FLOOR</code> operator applied to int values. */
-    public static int floor(int b0, int b1) {
-        int r = b0 % b1;
-        if (r < 0) {
-            r += b1;
-        }
-        return b0 - r;
-    }
-
-    /** SQL <code>FLOOR</code> operator applied to long values. */
-    public static long floor(long b0, long b1) {
-        long r = b0 % b1;
-        if (r < 0) {
-            r += b1;
-        }
-        return b0 - r;
-    }
-
-    // temporary
-    public static BigDecimal floor(BigDecimal b0, int b1) {
-        return floor(b0, BigDecimal.valueOf(b1));
-    }
-
-    // temporary
-    public static int floor(int b0, BigDecimal b1) {
-        return floor(b0, b1.intValue());
-    }
-
-    public static BigDecimal floor(BigDecimal b0, BigDecimal b1) {
-        final BigDecimal[] bigDecimals = b0.divideAndRemainder(b1);
-        BigDecimal r = bigDecimals[1];
-        if (r.signum() < 0) {
-            r = r.add(b1);
-        }
-        return b0.subtract(r);
-    }
-
-    // CEIL
-
-    public static double ceil(double b0) {
-        return Math.ceil(b0);
-    }
-
-    public static float ceil(float b0) {
-        return (float) Math.ceil(b0);
-    }
-
-    public static BigDecimal ceil(BigDecimal b0) {
-        return b0.setScale(0, BigDecimal.ROUND_CEILING);
-    }
-
-    /** SQL <code>CEIL</code> operator applied to byte values. */
-    public static byte ceil(byte b0, byte b1) {
-        return floor((byte) (b0 + b1 - 1), b1);
-    }
-
-    /** SQL <code>CEIL</code> operator applied to short values. */
-    public static short ceil(short b0, short b1) {
-        return floor((short) (b0 + b1 - 1), b1);
-    }
-
-    /** SQL <code>CEIL</code> operator applied to int values. */
-    public static int ceil(int b0, int b1) {
-        int r = b0 % b1;
-        if (r > 0) {
-            r -= b1;
-        }
-        return b0 - r;
-    }
-
-    /** SQL <code>CEIL</code> operator applied to long values. */
-    public static long ceil(long b0, long b1) {
-        return floor(b0 + b1 - 1, b1);
-    }
-
-    // temporary
-    public static BigDecimal ceil(BigDecimal b0, int b1) {
-        return ceil(b0, BigDecimal.valueOf(b1));
-    }
-
-    // temporary
-    public static int ceil(int b0, BigDecimal b1) {
-        return ceil(b0, b1.intValue());
-    }
-
-    public static BigDecimal ceil(BigDecimal b0, BigDecimal b1) {
-        final BigDecimal[] bigDecimals = b0.divideAndRemainder(b1);
-        BigDecimal r = bigDecimals[1];
-        if (r.signum() > 0) {
-            r = r.subtract(b1);
-        }
-        return b0.subtract(r);
-    }
-
-    // ABS
-
-    /** SQL <code>ABS</code> operator applied to byte values. */
-    public static byte abs(byte b0) {
-        return (byte) Math.abs(b0);
-    }
-
-    /** SQL <code>ABS</code> operator applied to short values. */
-    public static short abs(short b0) {
-        return (short) Math.abs(b0);
-    }
-
-    /** SQL <code>ABS</code> operator applied to int values. */
-    public static int abs(int b0) {
-        return Math.abs(b0);
-    }
-
-    /** SQL <code>ABS</code> operator applied to long values. */
-    public static long abs(long b0) {
-        return Math.abs(b0);
-    }
-
-    /** SQL <code>ABS</code> operator applied to float values. */
-    public static float abs(float b0) {
-        return Math.abs(b0);
-    }
-
-    /** SQL <code>ABS</code> operator applied to double values. */
-    public static double abs(double b0) {
-        return Math.abs(b0);
-    }
-
-    /** SQL <code>ABS</code> operator applied to BigDecimal values. */
-    public static BigDecimal abs(BigDecimal b0) {
-        return b0.abs();
-    }
-
-    // Helpers
-
-    /** Helper for implementing MIN. Somewhat similar to LEAST operator. */
-    public static <T extends Comparable<T>> T lesser(T b0, T b1) {
-        return b0 == null || b0.compareTo(b1) > 0 ? b1 : b0;
-    }
-
-    /** LEAST operator. */
-    public static <T extends Comparable<T>> T least(T b0, T b1) {
-        return b0 == null || b1 != null && b0.compareTo(b1) > 0 ? b1 : b0;
-    }
-
-    public static boolean greater(boolean b0, boolean b1) {
-        return b0 || b1;
-    }
-
-    public static boolean lesser(boolean b0, boolean b1) {
-        return b0 && b1;
-    }
-
-    public static byte greater(byte b0, byte b1) {
-        return b0 > b1 ? b0 : b1;
-    }
-
-    public static byte lesser(byte b0, byte b1) {
-        return b0 > b1 ? b1 : b0;
-    }
-
-    public static char greater(char b0, char b1) {
-        return b0 > b1 ? b0 : b1;
-    }
-
-    public static char lesser(char b0, char b1) {
-        return b0 > b1 ? b1 : b0;
-    }
-
-    public static short greater(short b0, short b1) {
-        return b0 > b1 ? b0 : b1;
-    }
-
-    public static short lesser(short b0, short b1) {
-        return b0 > b1 ? b1 : b0;
-    }
-
-    public static int greater(int b0, int b1) {
-        return b0 > b1 ? b0 : b1;
-    }
-
-    public static int lesser(int b0, int b1) {
-        return b0 > b1 ? b1 : b0;
-    }
-
-    public static long greater(long b0, long b1) {
-        return b0 > b1 ? b0 : b1;
-    }
-
-    public static long lesser(long b0, long b1) {
-        return b0 > b1 ? b1 : b0;
-    }
-
-    public static float greater(float b0, float b1) {
-        return b0 > b1 ? b0 : b1;
-    }
-
-    public static float lesser(float b0, float b1) {
-        return b0 > b1 ? b1 : b0;
-    }
-
-    public static double greater(double b0, double b1) {
-        return b0 > b1 ? b0 : b1;
-    }
-
-    public static double lesser(double b0, double b1) {
-        return b0 > b1 ? b1 : b0;
-    }
-
-    /** Helper for implementing MAX. Somewhat similar to GREATEST operator. */
-    public static <T extends Comparable<T>> T greater(T b0, T b1) {
-        return b0 == null || b0.compareTo(b1) < 0 ? b1 : b0;
-    }
-
-    /** GREATEST operator. */
-    public static <T extends Comparable<T>> T greatest(T b0, T b1) {
-        return b0 == null || b1 != null && b0.compareTo(b1) < 0 ? b1 : b0;
-    }
-
-    /** Boolean comparison. */
-    public static int compare(boolean x, boolean y) {
-        return x == y ? 0 : x ? 1 : -1;
-    }
-
-    /** CAST(FLOAT AS VARCHAR). */
-    public static String toString(float x) {
-        if (x == 0) {
-            return "0E0";
-        }
-        BigDecimal bigDecimal = new BigDecimal(x, MathContext.DECIMAL32).stripTrailingZeros();
-        final String s = bigDecimal.toString();
-        return s.replaceAll("0*E", "E").replace("E+", "E");
-    }
-
-    /** CAST(DOUBLE AS VARCHAR). */
-    public static String toString(double x) {
-        if (x == 0) {
-            return "0E0";
-        }
-        BigDecimal bigDecimal = new BigDecimal(x, MathContext.DECIMAL64).stripTrailingZeros();
-        final String s = bigDecimal.toString();
-        return s.replaceAll("0*E", "E").replace("E+", "E");
-    }
-
-    /** CAST(DECIMAL AS VARCHAR). */
-    public static String toString(BigDecimal x) {
-        final String s = x.toString();
-        if (s.startsWith("0")) {
-            // we want ".1" not "0.1"
-            return s.substring(1);
-        } else if (s.startsWith("-0")) {
-            // we want "-.1" not "-0.1"
-            return "-" + s.substring(2);
-        } else {
-            return s;
-        }
-    }
-
-    /** CAST(BOOLEAN AS VARCHAR). */
-    public static String toString(boolean x) {
-        // Boolean.toString returns lower case -- no good.
-        return x ? "TRUE" : "FALSE";
-    }
-
-    @NonDeterministic
-    private static Object cannotConvert(Object o, Class toType) {
-        throw new RuntimeException("Cannot convert " + o + " to " + toType);
-    }
-
-    /** CAST(VARCHAR AS BOOLEAN). */
-    public static boolean toBoolean(String s) {
-        s = trim_(s, true, true, ' ');
-        if (s.equalsIgnoreCase("TRUE")) {
-            return true;
-        } else if (s.equalsIgnoreCase("FALSE")) {
-            return false;
-        } else {
-            throw new RuntimeException("Invalid character for cast");
-        }
-    }
-
-    public static boolean toBoolean(Number number) {
-        return !number.equals(0);
-    }
-
-    public static boolean toBoolean(Object o) {
-        return o instanceof Boolean ? (Boolean) o : o instanceof Number ? toBoolean((Number) o) : o instanceof String ? toBoolean((String) o) : (Boolean) cannotConvert(o, boolean.class);
-    }
-
-    // Don't need parseByte etc. - Byte.parseByte is sufficient.
-
-    public static byte toByte(Object o) {
-        return o instanceof Byte ? (Byte) o : o instanceof Number ? toByte((Number) o) : Byte.parseByte(o.toString());
-    }
-
-    public static byte toByte(Number number) {
-        return number.byteValue();
-    }
-
-    public static char toChar(String s) {
-        return s.charAt(0);
-    }
-
-    public static Character toCharBoxed(String s) {
-        return s.charAt(0);
-    }
-
-    public static short toShort(String s) {
-        return Short.parseShort(s.trim());
-    }
-
-    public static short toShort(Number number) {
-        return number.shortValue();
-    }
-
-    public static short toShort(Object o) {
-        return o instanceof Short ? (Short) o : o instanceof Number ? toShort((Number) o) : o instanceof String ? toShort((String) o) : (Short) cannotConvert(o, short.class);
-    }
-
-    public static int toInt(java.util.Date v) {
-        return toInt(v, LOCAL_TZ);
-    }
-
-    public static int toInt(java.util.Date v, TimeZone timeZone) {
-        return (int) (toLong(v, timeZone) / DateTimeUtils.MILLIS_PER_DAY);
-    }
-
-    public static Integer toIntOptional(java.util.Date v) {
-        return v == null ? null : toInt(v);
-    }
-
-    public static Integer toIntOptional(java.util.Date v, TimeZone timeZone) {
-        return v == null ? null : toInt(v, timeZone);
-    }
-
-    public static long toLong(Date v) {
-        return toLong(v, LOCAL_TZ);
-    }
-
-    public static int toInt(java.sql.Time v) {
-        return (int) (toLong(v) % DateTimeUtils.MILLIS_PER_DAY);
-    }
-
-    public static Integer toIntOptional(java.sql.Time v) {
-        return v == null ? null : toInt(v);
-    }
-
-    public static int toInt(String s) {
-        return Integer.parseInt(s.trim());
-    }
-
-    public static int toInt(Number number) {
-        return number.intValue();
-    }
-
-    public static int toInt(Object o) {
-        return o instanceof Integer ? (Integer) o : o instanceof Number ? toInt((Number) o) : o instanceof String ? toInt((String) o) : (Integer) cannotConvert(o, int.class);
-    }
-
-    public static long toLong(Timestamp v) {
-        return toLong(v, LOCAL_TZ);
-    }
-
-    // mainly intended for java.sql.Timestamp but works for other dates also
-    public static long toLong(java.util.Date v, TimeZone timeZone) {
-        final long time = v.getTime();
-        return time + timeZone.getOffset(time);
-    }
-
-    // mainly intended for java.sql.Timestamp but works for other dates also
-    public static Long toLongOptional(java.util.Date v) {
-        return v == null ? null : toLong(v, LOCAL_TZ);
-    }
-
-    public static Long toLongOptional(Timestamp v, TimeZone timeZone) {
-        if (v == null) {
-            return null;
-        }
-        return toLong(v, LOCAL_TZ);
-    }
-
-    public static long toLong(String s) {
-        if (s.startsWith("199") && s.contains(":")) {
-            return Timestamp.valueOf(s).getTime();
-        }
-        return Long.parseLong(s.trim());
-    }
-
-    public static long toLong(Number number) {
-        return number.longValue();
-    }
-
-    public static long toLong(Object o) {
-        return o instanceof Long ? (Long) o : o instanceof Number ? toLong((Number) o) : o instanceof String ? toLong((String) o) : (Long) cannotConvert(o, long.class);
-    }
-
-    public static float toFloat(String s) {
-        return Float.parseFloat(s.trim());
-    }
-
-    public static float toFloat(Number number) {
-        return number.floatValue();
-    }
-
-    public static float toFloat(Object o) {
-        return o instanceof Float ? (Float) o : o instanceof Number ? toFloat((Number) o) : o instanceof String ? toFloat((String) o) : (Float) cannotConvert(o, float.class);
-    }
-
-    public static double toDouble(String s) {
-        return Double.parseDouble(s.trim());
-    }
-
-    public static double toDouble(Number number) {
-        return number.doubleValue();
-    }
-
-    public static double toDouble(Object o) {
-        return o instanceof Double ? (Double) o : o instanceof Number ? toDouble((Number) o) : o instanceof String ? toDouble((String) o) : (Double) cannotConvert(o, double.class);
-    }
-
-    public static BigDecimal toBigDecimal(String s) {
-        return new BigDecimal(s.trim());
-    }
-
-    public static BigDecimal toBigDecimal(Number number) {
-        // There are some values of "long" that cannot be represented as "double".
-        // Not so "int". If it isn't a long, go straight to double.
-        return number instanceof BigDecimal ? (BigDecimal) number : number instanceof BigInteger ? new BigDecimal((BigInteger) number) : number instanceof Long ? new BigDecimal(number.longValue()) : new BigDecimal(number.doubleValue());
-    }
-
-    public static BigDecimal toBigDecimal(Object o) {
-        return o instanceof Number ? toBigDecimal((Number) o) : toBigDecimal(o.toString());
-    }
-
-    // Don't need shortValueOf etc. - Short.valueOf is sufficient.
-
-    /** Helper for CAST(... AS VARCHAR(maxLength)). */
-    public static String truncate(String s, int maxLength) {
-        return s == null ? null : s.length() > maxLength ? s.substring(0, maxLength) : s;
-    }
-
-    /** Helper for CAST(... AS VARBINARY(maxLength)). */
-    public static ByteString truncate(ByteString s, int maxLength) {
-        return s == null ? null : s.length() > maxLength ? s.substring(0, maxLength) : s;
-    }
-
-    /** SQL {@code POSITION(seek IN string)} function. */
-    public static int position(String seek, String s) {
-        return s.indexOf(seek) + 1;
-    }
-
-    /** SQL {@code POSITION(seek IN string)} function. */
-    public static int position(ByteString seek, ByteString s) {
-        return s.indexOf(seek) + 1;
-    }
-
-    /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
-    public static long round(long v, long x) {
-        return truncate(v + x / 2, x);
-    }
-
-    /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
-    public static long truncate(long v, long x) {
-        long remainder = v % x;
-        if (remainder < 0) {
-            remainder += x;
-        }
-        return v - remainder;
-    }
-
-    /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
-    public static int round(int v, int x) {
-        return truncate(v + x / 2, x);
-    }
-
-    /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
-    public static int truncate(int v, int x) {
-        int remainder = v % x;
-        if (remainder < 0) {
-            remainder += x;
-        }
-        return v - remainder;
-    }
-
-    /** SQL {@code CURRENT_TIMESTAMP} function. */
-    @NonDeterministic
-    public static long currentTimestamp(DataContext root) {
-        // Cast required for JDK 1.6.
-        return (Long) DataContext.Variable.CURRENT_TIMESTAMP.get(root);
-    }
-
-    /** SQL {@code CURRENT_TIME} function. */
-    @NonDeterministic
-    public static int currentTime(DataContext root) {
-        int time = (int) (currentTimestamp(root) % DateTimeUtils.MILLIS_PER_DAY);
-        if (time < 0) {
-            time += DateTimeUtils.MILLIS_PER_DAY;
-        }
-        return time;
-    }
-
-    /** SQL {@code CURRENT_DATE} function. */
-    @NonDeterministic
-    public static int currentDate(DataContext root) {
-        final long timestamp = currentTimestamp(root);
-        int date = (int) (timestamp / DateTimeUtils.MILLIS_PER_DAY);
-        final int time = (int) (timestamp % DateTimeUtils.MILLIS_PER_DAY);
-        if (time < 0) {
-            --date;
-        }
-        return date;
-    }
-
-    /** SQL {@code LOCAL_TIMESTAMP} function. */
-    @NonDeterministic
-    public static long localTimestamp(DataContext root) {
-        // Cast required for JDK 1.6.
-        return (Long) DataContext.Variable.LOCAL_TIMESTAMP.get(root);
-    }
-
-    /** SQL {@code LOCAL_TIME} function. */
-    @NonDeterministic
-    public static int localTime(DataContext root) {
-        return (int) (localTimestamp(root) % DateTimeUtils.MILLIS_PER_DAY);
-    }
-
-    /** Helper for "array element reference". Caller has already ensured that
-     * array and index are not null. Index is 1-based, per SQL. */
-    public static Object arrayItem(List list, int item) {
-        if (item < 1 || item > list.size()) {
-            return null;
-        }
-        return list.get(item - 1);
-    }
-
-    /** Helper for "map element reference". Caller has already ensured that
-     * array and index are not null. Index is 1-based, per SQL. */
-    public static Object mapItem(Map map, Object item) {
-        return map.get(item);
-    }
-
-    /** Implements the {@code [ ... ]} operator on an object whose type is not
-     * known until runtime.
-     */
-    public static Object item(Object object, Object index) {
-        if (object instanceof Map) {
-            return ((Map) object).get(index);
-        }
-        if (object instanceof List && index instanceof Number) {
-            List list = (List) object;
-            return list.get(((Number) index).intValue());
-        }
-        return null;
-    }
-
-    /** NULL &rarr; FALSE, FALSE &rarr; FALSE, TRUE &rarr; TRUE. */
-    public static boolean isTrue(Boolean b) {
-        return b != null && b;
-    }
-
-    /** NULL &rarr; TRUE, FALSE &rarr; FALSE, TRUE &rarr; TRUE. */
-    public static boolean isNotFalse(Boolean b) {
-        return b == null || b;
-    }
-
-    /** NULL &rarr; NULL, FALSE &rarr; TRUE, TRUE &rarr; FALSE. */
-    public static Boolean not(Boolean b) {
-        return (b == null) ? null : !b;
-    }
-
-    /** Converts a JDBC array to a list. */
-    public static List arrayToList(final java.sql.Array a) {
-        if (a == null) {
-            return null;
-        }
-        try {
-            return Primitive.asList(a.getArray());
-        } catch (SQLException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /** Support the {@code CURRENT VALUE OF sequence} operator. */
-    @NonDeterministic
-    public static long sequenceCurrentValue(String key) {
-        return getAtomicLong(key).get();
-    }
-
-    /** Support the {@code NEXT VALUE OF sequence} operator. */
-    @NonDeterministic
-    public static long sequenceNextValue(String key) {
-        return getAtomicLong(key).incrementAndGet();
-    }
-
-    private static AtomicLong getAtomicLong(String key) {
-        final Map<String, AtomicLong> map = THREAD_SEQUENCES.get();
-        AtomicLong atomic = map.get(key);
-        if (atomic == null) {
-            atomic = new AtomicLong();
-            map.put(key, atomic);
-        }
-        return atomic;
-    }
-
-    /** Support the SLICE function. */
-    public static List slice(List list) {
-        return list;
-    }
-
-    /** Support the ELEMENT function. */
-    public static Object element(List list) {
-        switch (list.size()) {
-        case 0:
-            return null;
-        case 1:
-            return list.get(0);
-        default:
-            throw new RuntimeException("more than one value");
-        }
-    }
-
-    /** Returns a lambda that converts a list to an enumerable. */
-    public static <E> Function1<List<E>, Enumerable<E>> listToEnumerable() {
-        //noinspection unchecked
-        return (Function1<List<E>, Enumerable<E>>) (Function1) LIST_AS_ENUMERABLE;
-    }
-
-}
-
-// End SqlFunctions.java

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/80fbd77e/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 0893408..c184f0b 100644
--- a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -15,26 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.calcite.sql2rel;
-
-import static org.apache.calcite.sql.SqlUtil.stripAs;
-import static org.apache.calcite.util.Static.RESOURCE;
 
-import java.lang.reflect.Type;
-import java.math.BigDecimal;
-import java.util.AbstractList;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.Stack;
-import java.util.TreeSet;
-import java.util.logging.Level;
-import java.util.logging.Logger;
+package org.apache.calcite.sql2rel;
 
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.Convention;
@@ -58,6 +40,7 @@ import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.Sample;
 import org.apache.calcite.rel.core.Uncollect;
 import org.apache.calcite.rel.logical.LogicalAggregate;
@@ -161,7 +144,6 @@ import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.NumberUtil;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
-import org.apache.calcite.util.mapping.Mappings;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.base.Function;
@@ -173,6 +155,25 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Stack;
+import java.util.TreeSet;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import static org.apache.calcite.sql.SqlUtil.stripAs;
+import static org.apache.calcite.util.Static.RESOURCE;
+
 /*
  * OVERRIDE POINT:
  * - getInSubqueryThreshold(), was `20`, now `Integer.MAX_VALUE`
@@ -1449,6 +1450,12 @@ public class SqlToRelConverter {
         SqlCall aggCall = call.operand(0);
         SqlNode windowOrRef = call.operand(1);
         final SqlWindow window = validator.resolveWindow(windowOrRef, bb.scope, true);
+        // ROW_NUMBER() expects specific kind of framing.
+        if (aggCall.getOperator() == SqlStdOperatorTable.ROW_NUMBER) {
+            window.setLowerBound(SqlWindow.createUnboundedPreceding(SqlParserPos.ZERO));
+            window.setUpperBound(SqlWindow.createCurrentRow(SqlParserPos.ZERO));
+            window.setRows(SqlLiteral.createBoolean(true, SqlParserPos.ZERO));
+        }
         final SqlNodeList partitionList = window.getPartitionList();
         final ImmutableList.Builder<RexNode> partitionKeys = ImmutableList.builder();
         for (SqlNode partition : partitionList) {
@@ -1577,11 +1584,15 @@ public class SqlToRelConverter {
             RelNode rightRel = rightBlackboard.root;
             JoinRelType convertedJoinType = convertJoinType(joinType);
             RexNode conditionExp;
+            final SqlValidatorNamespace leftNamespace = validator.getNamespace(left);
+            final SqlValidatorNamespace rightNamespace = validator.getNamespace(right);
             if (isNatural) {
-                final List<String> columnList = SqlValidatorUtil.deriveNaturalJoinColumnList(validator.getNamespace(left).getRowType(), validator.getNamespace(right).getRowType());
-                conditionExp = convertUsing(leftRel, rightRel, columnList);
+                final RelDataType leftRowType = leftNamespace.getRowType();
+                final RelDataType rightRowType = rightNamespace.getRowType();
+                final List<String> columnList = SqlValidatorUtil.deriveNaturalJoinColumnList(leftRowType, rightRowType);
+                conditionExp = convertUsing(leftNamespace, rightNamespace, columnList);
             } else {
-                conditionExp = convertJoinCondition(fromBlackboard, join.getCondition(), join.getConditionType(), leftRel, rightRel);
+                conditionExp = convertJoinCondition(fromBlackboard, leftNamespace, rightNamespace, join.getCondition(), join.getConditionType(), leftRel, rightRel);
             }
 
             final RelNode joinRel = createJoin(fromBlackboard, leftRel, rightRel, conditionExp, convertedJoinType);
@@ -1777,58 +1788,9 @@ public class SqlToRelConverter {
             }
         }
 
-        final List<RexNode> extraLeftExprs = new ArrayList<>();
-        final List<RexNode> extraRightExprs = new ArrayList<>();
-        final int leftCount = leftRel.getRowType().getFieldCount();
-        final int rightCount = rightRel.getRowType().getFieldCount();
-        if (!containsGet(joinCond)) {
-            joinCond = pushDownJoinConditions(joinCond, leftCount, rightCount, extraLeftExprs, extraRightExprs);
-        }
-        if (!extraLeftExprs.isEmpty()) {
-            final List<RelDataTypeField> fields = leftRel.getRowType().getFieldList();
-            leftRel = RelOptUtil.createProject(leftRel, new AbstractList<Pair<RexNode, String>>() {
-                @Override
-                public int size() {
-                    return leftCount + extraLeftExprs.size();
-                }
-
-                @Override
-                public Pair<RexNode, String> get(int index) {
-                    if (index < leftCount) {
-                        RelDataTypeField field = fields.get(index);
-                        return Pair.<RexNode, String> of(new RexInputRef(index, field.getType()), field.getName());
-                    } else {
-                        return Pair.of(extraLeftExprs.get(index - leftCount), null);
-                    }
-                }
-            }, true);
-        }
-        if (!extraRightExprs.isEmpty()) {
-            final List<RelDataTypeField> fields = rightRel.getRowType().getFieldList();
-            final int newLeftCount = leftCount + extraLeftExprs.size();
-            rightRel = RelOptUtil.createProject(rightRel, new AbstractList<Pair<RexNode, String>>() {
-                @Override
-                public int size() {
-                    return rightCount + extraRightExprs.size();
-                }
+        final Join originalJoin = (Join) RelFactories.DEFAULT_JOIN_FACTORY.createJoin(leftRel, rightRel, joinCond, joinType, ImmutableSet.<String> of(), false);
 
-                @Override
-                public Pair<RexNode, String> get(int index) {
-                    if (index < rightCount) {
-                        RelDataTypeField field = fields.get(index);
-                        return Pair.<RexNode, String> of(new RexInputRef(index, field.getType()), field.getName());
-                    } else {
-                        return Pair.of(RexUtil.shift(extraRightExprs.get(index - rightCount), -newLeftCount), null);
-                    }
-                }
-            }, true);
-        }
-        RelNode join = createJoin(leftRel, rightRel, joinCond, joinType, ImmutableSet.<String> of());
-        if (!extraLeftExprs.isEmpty() || !extraRightExprs.isEmpty()) {
-            Mappings.TargetMapping mapping = Mappings.createShiftMapping(leftCount + extraLeftExprs.size() + rightCount + extraRightExprs.size(), 0, 0, leftCount, leftCount, leftCount + extraLeftExprs.size(), rightCount);
-            return RelOptUtil.createProject(join, mapping);
-        }
-        return join;
+        return RelOptUtil.pushDownJoinConditions(originalJoin);
     }
 
     private static boolean containsGet(RexNode node) {
@@ -1849,92 +1811,6 @@ public class SqlToRelConverter {
     }
 
     /**
-     * Pushes down parts of a join condition. For example, given
-     * "emp JOIN dept ON emp.deptno + 1 = dept.deptno", adds a project above
-     * "emp" that computes the expression
-     * "emp.deptno + 1". The resulting join condition is a simple combination
-     * of AND, equals, and input fields.
-     */
-    private RexNode pushDownJoinConditions(RexNode node, int leftCount, int rightCount, List<RexNode> extraLeftExprs, List<RexNode> extraRightExprs) {
-        switch (node.getKind()) {
-        case AND:
-        case OR:
-        case EQUALS:
-            final RexCall call = (RexCall) node;
-            final List<RexNode> list = new ArrayList<>();
-            List<RexNode> operands = Lists.newArrayList(call.getOperands());
-            for (int i = 0; i < operands.size(); i++) {
-                RexNode operand = operands.get(i);
-                final int left2 = leftCount + extraLeftExprs.size();
-                final int right2 = rightCount + extraRightExprs.size();
-                final RexNode e = pushDownJoinConditions(operand, leftCount, rightCount, extraLeftExprs, extraRightExprs);
-                final List<RexNode> remainingOperands = Util.skip(operands, i + 1);
-                final int left3 = leftCount + extraLeftExprs.size();
-                final int right3 = rightCount + extraRightExprs.size();
-                fix(remainingOperands, left2, left3);
-                fix(list, left2, left3);
-                list.add(e);
-            }
-            if (!list.equals(call.getOperands())) {
-                return call.clone(call.getType(), list);
-            }
-            return call;
-        case INPUT_REF:
-        case LITERAL:
-            return node;
-        default:
-            ImmutableBitSet bits = RelOptUtil.InputFinder.bits(node);
-            final int mid = leftCount + extraLeftExprs.size();
-            switch (Side.of(bits, mid)) {
-            case LEFT:
-                fix(extraRightExprs, mid, mid + 1);
-                extraLeftExprs.add(node);
-                return new RexInputRef(mid, node.getType());
-            case RIGHT:
-                final int index2 = mid + rightCount + extraRightExprs.size();
-                extraRightExprs.add(node);
-                return new RexInputRef(index2, node.getType());
-            case BOTH:
-            case EMPTY:
-            default:
-                return node;
-            }
-        }
-    }
-
-    private void fix(List<RexNode> operands, int before, int after) {
-        if (before == after) {
-            return;
-        }
-        for (int i = 0; i < operands.size(); i++) {
-            RexNode node = operands.get(i);
-            operands.set(i, RexUtil.shift(node, before, after - before));
-        }
-    }
-
-    /**
-     * Categorizes whether a bit set contains bits left and right of a
-     * line.
-     */
-    enum Side {
-        LEFT, RIGHT, BOTH, EMPTY;
-
-        static Side of(ImmutableBitSet bitSet, int middle) {
-            final int firstBit = bitSet.nextSetBit(0);
-            if (firstBit < 0) {
-                return EMPTY;
-            }
-            if (firstBit >= middle) {
-                return RIGHT;
-            }
-            if (bitSet.nextSetBit(middle) < 0) {
-                return LEFT;
-            }
-            return BOTH;
-        }
-    }
-
-    /**
      * Determines whether a subquery is non-correlated. Note that a
      * non-correlated subquery can contain correlated references, provided those
      * references do not reference select statements that are parents of the
@@ -1986,7 +1862,7 @@ public class SqlToRelConverter {
         return Collections.emptyList();
     }
 
-    private RexNode convertJoinCondition(Blackboard bb, SqlNode condition, JoinConditionType conditionType, RelNode leftRel, RelNode rightRel) {
+    private RexNode convertJoinCondition(Blackboard bb, SqlValidatorNamespace leftNamespace, SqlValidatorNamespace rightNamespace, SqlNode condition, JoinConditionType conditionType, RelNode leftRel, RelNode rightRel) {
         if (condition == null) {
             return rexBuilder.makeLiteral(true);
         }
@@ -2004,7 +1880,7 @@ public class SqlToRelConverter {
                 String name = id.getSimple();
                 nameList.add(name);
             }
-            return convertUsing(leftRel, rightRel, nameList);
+            return convertUsing(leftNamespace, rightNamespace, nameList);
         default:
             throw Util.unexpected(conditionType);
         }
@@ -2015,23 +1891,24 @@ public class SqlToRelConverter {
      * from NATURAL JOIN. "a JOIN b USING (x, y)" becomes "a.x = b.x AND a.y =
      * b.y". Returns null if the column list is empty.
      *
-     * @param leftRel  Left input to the join
-     * @param rightRel Right input to the join
+     * @param leftNamespace Namespace of left input to join
+     * @param rightNamespace Namespace of right input to join
      * @param nameList List of column names to join on
      * @return Expression to match columns from name list, or true if name list
      * is empty
      */
-    private RexNode convertUsing(RelNode leftRel, RelNode rightRel, List<String> nameList) {
+    private RexNode convertUsing(SqlValidatorNamespace leftNamespace, SqlValidatorNamespace rightNamespace, List<String> nameList) {
         final List<RexNode> list = Lists.newArrayList();
         for (String name : nameList) {
-            final RelDataType leftRowType = leftRel.getRowType();
-            RelDataTypeField leftField = catalogReader.field(leftRowType, name);
-            RexNode left = rexBuilder.makeInputRef(leftField.getType(), leftField.getIndex());
-            final RelDataType rightRowType = rightRel.getRowType();
-            RelDataTypeField rightField = catalogReader.field(rightRowType, name);
-            RexNode right = rexBuilder.makeInputRef(rightField.getType(), leftRowType.getFieldList().size() + rightField.getIndex());
-            RexNode equalsCall = rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, left, right);
-            list.add(equalsCall);
+            List<RexNode> operands = new ArrayList<>();
+            int offset = 0;
+            for (SqlValidatorNamespace n : ImmutableList.of(leftNamespace, rightNamespace)) {
+                final RelDataType rowType = n.getRowType();
+                final RelDataTypeField field = catalogReader.field(rowType, name);
+                operands.add(rexBuilder.makeInputRef(field.getType(), offset + field.getIndex()));
+                offset += rowType.getFieldList().size();
+            }
+            list.add(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, operands));
         }
         return RexUtil.composeConjunction(rexBuilder, list, false);
     }
@@ -2866,6 +2743,7 @@ public class SqlToRelConverter {
         for (int i = 0; i < joinList.size(); i++) {
             Object o = joinList.get(i);
             if (o instanceof List) {
+                @SuppressWarnings("unchecked")
                 List<SqlNode> projectList = (List<SqlNode>) o;
                 final List<RexNode> selectList = new ArrayList<>();
                 final List<String> fieldNameList = new ArrayList<>();
@@ -2892,28 +2770,11 @@ public class SqlToRelConverter {
         RelNode ret = (RelNode) joinList.get(0);
         for (int i = 1; i < joinList.size(); i++) {
             RelNode relNode = (RelNode) joinList.get(i);
-            ret = createJoin(ret, relNode, rexBuilder.makeLiteral(true), JoinRelType.INNER, ImmutableSet.<String> of());
+            ret = RelFactories.DEFAULT_JOIN_FACTORY.createJoin(ret, relNode, rexBuilder.makeLiteral(true), JoinRelType.INNER, ImmutableSet.<String> of(), false);
         }
         return ret;
     }
 
-    /**
-     * Factory method that creates a join.
-     * A subclass can override to use a different kind of join.
-     *
-     * @param left             Left input
-     * @param right            Right input
-     * @param condition        Join condition
-     * @param joinType         Join type
-     * @param variablesStopped Set of names of variables which are set by the
-     *                         LHS and used by the RHS and are not available to
-     *                         nodes above this LogicalJoin in the tree
-     * @return A relational expression representing a join
-     */
-    protected RelNode createJoin(RelNode left, RelNode right, RexNode condition, JoinRelType joinType, Set<String> variablesStopped) {
-        return LogicalJoin.create(left, right, condition, joinType, variablesStopped);
-    }
-
     private void convertSelectList(Blackboard bb, SqlSelect select, List<SqlNode> orderList) {
         SqlNodeList selectList = select.getSelectList();
         selectList = validator.expandStar(selectList, select, false);
@@ -3779,6 +3640,10 @@ public class SqlToRelConverter {
                 // for now do not detect aggregates in subqueries.
                 return null;
             }
+            // ignore window aggregates and ranking functions (associated with OVER operator)
+            if (call.getOperator().getKind() == SqlKind.OVER) {
+                return null;
+            }
             if (call.getOperator().isAggregator()) {
                 translateAgg(call, null, call);
                 return null;
@@ -4067,7 +3932,7 @@ public class SqlToRelConverter {
                     exprs.set(0, reinterpretCast ? rexBuilder.makeReinterpretCast(histogramType, exprs.get(0), rexBuilder.makeLiteral(false)) : rexBuilder.makeCast(histogramType, exprs.get(0)));
                 }
 
-                RexCallBinding bind = new RexCallBinding(rexBuilder.getTypeFactory(), SqlStdOperatorTable.HISTOGRAM_AGG, exprs);
+                RexCallBinding bind = new RexCallBinding(rexBuilder.getTypeFactory(), SqlStdOperatorTable.HISTOGRAM_AGG, exprs, ImmutableList.<RelCollation> of());
 
                 RexNode over = rexBuilder.makeOver(SqlStdOperatorTable.HISTOGRAM_AGG.inferReturnType(bind), SqlStdOperatorTable.HISTOGRAM_AGG, exprs, partitionKeys, orderKeys, lowerBound, upperBound, window.isRows(), window.isAllowPartial(), false);
 
@@ -4152,6 +4017,10 @@ public class SqlToRelConverter {
 
         @Override
         public Void visit(SqlCall call) {
+            // ignore window aggregates and ranking functions (associated with OVER operator)
+            if (call.getOperator().getKind() == SqlKind.OVER) {
+                return null;
+            }
             if (call.getOperator().isAggregator()) {
                 list.add(call);
                 return null;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/80fbd77e/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
index 49f58fd..79defcc 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
@@ -50,7 +50,7 @@ public class KylinMeta extends MetaImpl {
 
     // insert/update/delete go this path, ignorable for Kylin
     @Override
-    public StatementHandle prepare(ConnectionHandle ch, String sql, int maxRowCount) {
+    public StatementHandle prepare(ConnectionHandle ch, String sql, long maxRowCount) {
         StatementHandle result = super.createStatement(ch);
         result.signature = connection().mockPreparedSignature(sql);
         return result;
@@ -58,16 +58,15 @@ public class KylinMeta extends MetaImpl {
 
     // mimic from CalciteMetaImpl, real execution happens via callback in KylinResultSet.execute()
     @Override
-    public ExecuteResult prepareAndExecute(ConnectionHandle ch, String sql, int maxRowCount, PrepareCallback callback) {
-        final StatementHandle sh;
+    public ExecuteResult prepareAndExecute(StatementHandle sh, String sql, long maxRowCount, PrepareCallback callback) {
         try {
             synchronized (callback.getMonitor()) {
                 callback.clear();
-                sh = prepare(ch, sql, maxRowCount);
+                sh.signature = connection().mockPreparedSignature(sql);
                 callback.assign(sh.signature, null, -1);
             }
             callback.execute();
-            final MetaResultSet metaResultSet = MetaResultSet.create(ch.id, sh.id, false, sh.signature, null);
+            final MetaResultSet metaResultSet = MetaResultSet.create(sh.connectionId, sh.id, false, sh.signature, null);
             return new ExecuteResult(ImmutableList.of(metaResultSet));
         } catch (SQLException e) {
             throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/80fbd77e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e72c81f..b1df6e3 100644
--- a/pom.xml
+++ b/pom.xml
@@ -85,7 +85,7 @@
         <spring.framework.version>3.1.2.RELEASE</spring.framework.version>
 
         <!-- Calcite Version -->
-        <calcite.version>1.3.0-incubating</calcite.version>
+        <calcite.version>1.4.0-incubating</calcite.version>
 
         <!-- Curator.version Version -->
         <curator.version>2.6.0</curator.version>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/80fbd77e/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
index 2b3d3e3..30f059c 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
@@ -18,6 +18,7 @@
 
 package org.apache.kylin.query.relnode;
 
+import java.lang.reflect.Constructor;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -226,13 +227,32 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
         }
     }
 
+    // workaround that EnumerableJoin constructor is protected
+    private static Constructor<EnumerableJoin> constr;
+    static {
+        try {
+            constr = EnumerableJoin.class.getDeclaredConstructor(RelOptCluster.class, //
+                    RelTraitSet.class, //
+                    RelNode.class, //
+                    RelNode.class, //
+                    RexNode.class, //
+                    ImmutableIntList.class, //
+                    ImmutableIntList.class, //
+                    JoinRelType.class, //
+                    Set.class);
+            constr.setAccessible(true);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
     @Override
     public EnumerableRel implementEnumerable(List<EnumerableRel> inputs) {
         if (this.hasSubQuery) {
             try {
-                return new EnumerableJoin(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), //
+                return constr.newInstance(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), //
                         inputs.get(0), inputs.get(1), condition, leftKeys, rightKeys, joinType, variablesStopped);
-            } catch (InvalidRelException e) {
+            } catch (Exception e) {
                 throw new IllegalStateException("Can't create EnumerableJoin!", e);
             }
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/80fbd77e/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
index b1d8d1a..d2661b4 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
@@ -47,6 +47,7 @@ import org.apache.calcite.rel.rules.FilterJoinRule;
 import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
 import org.apache.calcite.rel.rules.JoinCommuteRule;
 import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
+import org.apache.calcite.rel.rules.ReduceExpressionsRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -123,11 +124,21 @@ public class OLAPTableScan extends TableScan implements OLAPRel, EnumerableRel {
         planner.addRule(OLAPLimitRule.INSTANCE);
         planner.addRule(OLAPSortRule.INSTANCE);
 
+        // CalcitePrepareImpl.CONSTANT_REDUCTION_RULES
+        planner.addRule(ReduceExpressionsRule.PROJECT_INSTANCE);
+        planner.addRule(ReduceExpressionsRule.FILTER_INSTANCE);
+        planner.addRule(ReduceExpressionsRule.CALC_INSTANCE);
+        planner.addRule(ReduceExpressionsRule.JOIN_INSTANCE);
+        // the ValuesReduceRule breaks query test somehow...
+        //        planner.addRule(ValuesReduceRule.FILTER_INSTANCE);
+        //        planner.addRule(ValuesReduceRule.PROJECT_FILTER_INSTANCE);
+        //        planner.addRule(ValuesReduceRule.PROJECT_INSTANCE);
+
         // since join is the entry point, we can't push filter past join
         planner.removeRule(FilterJoinRule.FILTER_ON_JOIN);
         planner.removeRule(FilterJoinRule.JOIN);
 
-        // TODO : since we don't have statistic of table, the optimization of join is too cost
+        // since we don't have statistic of table, the optimization of join is too cost
         planner.removeRule(JoinCommuteRule.INSTANCE);
         planner.removeRule(JoinPushThroughJoinRule.LEFT);
         planner.removeRule(JoinPushThroughJoinRule.RIGHT);


[38/45] incubator-kylin git commit: KYLIN-999 exclude “website” from release

Posted by sh...@apache.org.
KYLIN-999 exclude “website” from release

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

Branch: refs/heads/master
Commit: 1696f701a5003ee2746418954456dff079e5f6eb
Parents: 875ca21
Author: shaofengshi <sh...@apache.org>
Authored: Thu Oct 8 10:35:33 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Oct 8 10:35:33 2015 +0800

----------------------------------------------------------------------
 src/main/config/assemblies/source-assembly.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1696f701/src/main/config/assemblies/source-assembly.xml
----------------------------------------------------------------------
diff --git a/src/main/config/assemblies/source-assembly.xml b/src/main/config/assemblies/source-assembly.xml
index 96a56a1..87641de 100644
--- a/src/main/config/assemblies/source-assembly.xml
+++ b/src/main/config/assemblies/source-assembly.xml
@@ -72,7 +72,7 @@ limitations under the License.
                 <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.gitignore(/.*)?]
                 </exclude>
 
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?docs/website(/.*)?]
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?website(/.*)?]
                 </exclude>
 
                 <!-- release-plugin temp files -->


[36/45] incubator-kylin git commit: add powered by page

Posted by sh...@apache.org.
add powered by page


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

Branch: refs/heads/master
Commit: 5b98b7e9031485a7b929745dcc54a4f57a6f779b
Parents: 1776016
Author: lukehan <lu...@apache.org>
Authored: Wed Oct 7 23:58:08 2015 +0800
Committer: lukehan <lu...@apache.org>
Committed: Wed Oct 7 23:58:08 2015 +0800

----------------------------------------------------------------------
 website/community/poweredby.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5b98b7e9/website/community/poweredby.md
----------------------------------------------------------------------
diff --git a/website/community/poweredby.md b/website/community/poweredby.md
index 8312ac1..e600cec 100644
--- a/website/community/poweredby.md
+++ b/website/community/poweredby.md
@@ -14,5 +14,5 @@ __Companies__
 * [eBay](http://www.ebay.com), Kylin was developed inside eBay for Big Data Analytics on Hadoop and then contributed to open source world in Oct 2014, accepted as Apache Incubator project in Nov 2014. Apache Kylin is used at eBay for OLAP on Hadoop. This powers various data products including Behavior Analytics, Traffic Reporting, Account Manager Application and Streaming Dashboard.
 * [MiningLAMP](http://www.mininglamp.com/), Kylin is part of MiningLAMP's Big Data Analytics product offering OLAP capability 
 * [Meituan](http://www.meituan.com)
-* TBD...
+* 
 


[29/45] incubator-kylin git commit: KYLIN-913 Fix PK explicitly declared as derived dimension

Posted by sh...@apache.org.
KYLIN-913 Fix PK explicitly declared as derived dimension


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

Branch: refs/heads/master
Commit: 07119116989d346f33e39627a0828d7e031803d6
Parents: 459a41d
Author: Yang Li <li...@apache.org>
Authored: Mon Oct 5 21:31:48 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Mon Oct 5 21:31:48 2015 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/kylin/cube/model/CubeDesc.java  | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/07119116/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index 7f9252c..8fd2c4e 100644
--- a/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -562,6 +562,17 @@ public class CubeDesc extends RootPersistentEntity {
     private void initDerivedMap(TblColRef[] hostCols, DeriveType type, DimensionDesc dimension, TblColRef[] derivedCols, String[] extra) {
         if (hostCols.length == 0 || derivedCols.length == 0)
             throw new IllegalStateException("host/derived columns must not be empty");
+        
+        // Although FK derives PK automatically, user unaware of this can declare PK as derived dimension explicitly.
+        // In that case, derivedCols[] will contain a FK which is transformed from the PK by initDimensionColRef().
+        // Must drop FK from derivedCols[] before continue.
+        for (int i = 0; i < derivedCols.length; i++) {
+            if (ArrayUtils.contains(hostCols, derivedCols[i])) {
+                derivedCols = (TblColRef[]) ArrayUtils.remove(derivedCols, i);
+                extra = (String[]) ArrayUtils.remove(extra, i);
+                i--;
+            }
+        }
 
         Array<TblColRef> hostColArray = new Array<TblColRef>(hostCols);
         List<DeriveInfo> infoList = hostToDerivedMap.get(hostColArray);


[35/45] incubator-kylin git commit: Merge branch '1.x-staging' of https://git-wip-us.apache.org/repos/asf/incubator-kylin into 1.x-staging

Posted by sh...@apache.org.
Merge branch '1.x-staging' of https://git-wip-us.apache.org/repos/asf/incubator-kylin into 1.x-staging


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

Branch: refs/heads/master
Commit: 1776016def297bbc08014a02097242fd2a1390c7
Parents: 290f5ce 22917f5
Author: lukehan <lu...@apache.org>
Authored: Wed Oct 7 23:54:13 2015 +0800
Committer: lukehan <lu...@apache.org>
Committed: Wed Oct 7 23:54:13 2015 +0800

----------------------------------------------------------------------
 website/_dev/howto_contribute.md | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[17/45] incubator-kylin git commit: minor, add tool command in kylin.sh

Posted by sh...@apache.org.
minor, add tool command in kylin.sh


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

Branch: refs/heads/master
Commit: 9ac01db5ffc71d115991e1635d823406b5d68e52
Parents: 6adf4a1
Author: Li, Yang <ya...@ebay.com>
Authored: Mon Sep 28 13:49:52 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Mon Sep 28 16:47:51 2015 +0800

----------------------------------------------------------------------
 bin/kylin.sh | 19 +++++++++++++++++++
 1 file changed, 19 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9ac01db5/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/bin/kylin.sh b/bin/kylin.sh
index ea84f22..1334ed1 100644
--- a/bin/kylin.sh
+++ b/bin/kylin.sh
@@ -21,6 +21,7 @@ dir=$(dirname ${0})
 source ${dir}/check-env.sh
 mkdir -p ${KYLIN_HOME}/logs
 
+# start command
 if [ $1 == "start" ]
 then
 
@@ -80,6 +81,8 @@ then
     fi
     echo "You can check the log at ${tomcat_root}/logs/kylin.log"
     exit 0
+
+# stop command
 elif [ $1 == "stop" ]
 then
     if [ ! -f "${KYLIN_HOME}/pid" ]
@@ -98,6 +101,22 @@ then
     fi
     rm ${KYLIN_HOME}/pid
     exit 0
+
+# tool command
+elif [[ $1 = org.apache.kylin.* ]]
+then
+    #retrive $hive_dependency and $hbase_dependency
+    source ${dir}/find-hive-dependency.sh
+    source ${dir}/find-hbase-dependency.sh
+    #retrive $KYLIN_EXTRA_START_OPTS
+    if [ -f "${dir}/setenv-tool.sh" ]
+        then source ${dir}/setenv-tool.sh
+    fi
+
+    export HBASE_CLASSPATH=${KYLIN_HOME}/lib/*:$hive_dependency:${HBASE_CLASSPATH}
+
+    exec hbase "$@"
+
 else
     echo "usage: kylin.sh start or kylin.sh stop"
     exit 1


[28/45] incubator-kylin git commit: KYLIN-457 Add missing file

Posted by sh...@apache.org.
KYLIN-457 Add missing file


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

Branch: refs/heads/master
Commit: 459a41d66e5dc5c675b08ae06b50654022ca1b78
Parents: 24e59fc
Author: Yang Li <li...@apache.org>
Authored: Sun Oct 4 20:56:41 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sun Oct 4 20:56:41 2015 +0800

----------------------------------------------------------------------
 .../kylin/rest/util/Log4jConfigListener.java    | 46 ++++++++++++++++++++
 1 file changed, 46 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/459a41d6/server/src/main/java/org/apache/kylin/rest/util/Log4jConfigListener.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/util/Log4jConfigListener.java b/server/src/main/java/org/apache/kylin/rest/util/Log4jConfigListener.java
new file mode 100644
index 0000000..19c76ea
--- /dev/null
+++ b/server/src/main/java/org/apache/kylin/rest/util/Log4jConfigListener.java
@@ -0,0 +1,46 @@
+/*
+ * 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.kylin.rest.util;
+
+import javax.servlet.ServletContextEvent;
+
+public class Log4jConfigListener extends org.springframework.web.util.Log4jConfigListener {
+
+    private boolean isTesting;
+
+    public Log4jConfigListener() {
+        // set by DebugTomcat
+        this.isTesting = "testing".equals(System.getProperty("spring.profiles.active"));
+    }
+
+    @Override
+    public void contextInitialized(ServletContextEvent event) {
+        if (!isTesting) {
+            super.contextInitialized(event);
+        }
+    }
+
+    @Override
+    public void contextDestroyed(ServletContextEvent event) {
+        if (!isTesting) {
+            super.contextDestroyed(event);
+        }
+    }
+
+}


[07/45] incubator-kylin git commit: KYLIN-1039 fix CI, add all joins in query81.sql

Posted by sh...@apache.org.
KYLIN-1039 fix CI, add all joins in query81.sql


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

Branch: refs/heads/master
Commit: ab7a363f28c681a24b68bf4a7a905dd39382dd6c
Parents: 038eb59
Author: Li, Yang <ya...@ebay.com>
Authored: Tue Sep 22 16:31:31 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Tue Sep 22 16:31:31 2015 +0800

----------------------------------------------------------------------
 query/src/test/resources/query/sql/query81.sql | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ab7a363f/query/src/test/resources/query/sql/query81.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query81.sql b/query/src/test/resources/query/sql/query81.sql
index 7302a7d..93868e7 100644
--- a/query/src/test/resources/query/sql/query81.sql
+++ b/query/src/test/resources/query/sql/query81.sql
@@ -18,6 +18,11 @@
 
 select test_cal_dt.week_beg_dt, sum(price) as GMV
  from test_kylin_fact 
- inner JOIN edw.test_cal_dt as test_cal_dt  ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
+inner JOIN edw.test_cal_dt as test_cal_dt
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ inner JOIN test_category_groupings
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id
+ inner JOIN edw.test_sites as test_sites
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id
  where test_cal_dt.week_beg_dt between DATE '2013-09-01' and DATE '2013-10-01' and (lstg_format_name='FP-GTC' or 'a' = 'b')
  group by test_cal_dt.week_beg_dt
\ No newline at end of file


[08/45] incubator-kylin git commit: css

Posted by sh...@apache.org.
css

Signed-off-by: Luke Han <lu...@apache.org>


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

Branch: refs/heads/master
Commit: ba57407467414213cd156dffaeaa5ee69f3bac42
Parents: ab7a363
Author: Luwei Chen <ch...@hotmail.com>
Authored: Fri Sep 18 16:03:59 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Wed Sep 23 19:05:20 2015 +0800

----------------------------------------------------------------------
 website/_includes/docs_nav.html   |  2 +-
 website/_includes/docs_ul.cn.html |  2 +-
 website/_includes/docs_ul.html    |  2 +-
 website/_layouts/docs-cn.html     |  2 ++
 website/_layouts/docs.html        |  5 +++--
 website/assets/css/docs.css       | 14 ++++++++++++--
 6 files changed, 20 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ba574074/website/_includes/docs_nav.html
----------------------------------------------------------------------
diff --git a/website/_includes/docs_nav.html b/website/_includes/docs_nav.html
index b4a4311..bec9648 100755
--- a/website/_includes/docs_nav.html
+++ b/website/_includes/docs_nav.html
@@ -30,4 +30,4 @@
     {% endfor %}
 
     </ul>
-</div>
\ No newline at end of file
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ba574074/website/_includes/docs_ul.cn.html
----------------------------------------------------------------------
diff --git a/website/_includes/docs_ul.cn.html b/website/_includes/docs_ul.cn.html
index 262159d..b42fc84 100644
--- a/website/_includes/docs_ul.cn.html
+++ b/website/_includes/docs_ul.cn.html
@@ -16,7 +16,7 @@
 
   {% for p in site.docs %}
     {% if p.url == item_url %}
-      <li><a href="{{ p.url }}" class="list-group-item-lay pjaxlink">{{p.title}}</a></li>      
+      <li><a href="{{ p.url }}" class="list-group-item-lay pjaxlink" id="navlist">{{p.title}}</a></li>      
       {% break %}
     {% endif %}
   {% endfor %}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ba574074/website/_includes/docs_ul.html
----------------------------------------------------------------------
diff --git a/website/_includes/docs_ul.html b/website/_includes/docs_ul.html
index d992aca..89e0805 100644
--- a/website/_includes/docs_ul.html
+++ b/website/_includes/docs_ul.html
@@ -17,7 +17,7 @@
 
   {% for p in site.docs %}
     {% if p.url == item_url %}
-      <li><a href="{{ p.url }}" class="list-group-item-lay pjaxlink">{{p.title}}</a></li>      
+      <li><a href="{{ p.url }}" class="list-group-item-lay pjaxlink" id="navlist">{{p.title}}</a></li>      
       {% break %}
     {% endif %}
   {% endfor %}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ba574074/website/_layouts/docs-cn.html
----------------------------------------------------------------------
diff --git a/website/_layouts/docs-cn.html b/website/_layouts/docs-cn.html
index 1c1baf6..5f13608 100755
--- a/website/_layouts/docs-cn.html
+++ b/website/_layouts/docs-cn.html
@@ -29,7 +29,9 @@
 					<div id="container">
 						<div id="pjax">
 							<h1 class="post-title">{{ page.title }}</h1>
+							<article class="post-content" >
 							{{ content }}
+							</article>
 						</div>
 					</div>
 				</div>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ba574074/website/_layouts/docs.html
----------------------------------------------------------------------
diff --git a/website/_layouts/docs.html b/website/_layouts/docs.html
index 51908b3..0aa9b63 100755
--- a/website/_layouts/docs.html
+++ b/website/_layouts/docs.html
@@ -32,9 +32,10 @@
 							{% if page.version == NULL %}
 							{% else %}							
 								<p>version: {{page.version}}, since: {{page.since}}</p>
-							{% endif %}	
-
+							{% endif %}
+							<article class="post-content" >	
 							{{ content }}
+							</article>
 						</div>
 					</div>
 				</div>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ba574074/website/assets/css/docs.css
----------------------------------------------------------------------
diff --git a/website/assets/css/docs.css b/website/assets/css/docs.css
index f824c8a..e5dbfc9 100755
--- a/website/assets/css/docs.css
+++ b/website/assets/css/docs.css
@@ -34,7 +34,7 @@
   font-weight: 600;
   font-family: 'Open sans';
   font-size: 15px;
-  padding: 10px 10px;
+  padding: 0px 30px 5px 0px;
 }
 
 .aside1> .nav > li > a:hover {
@@ -57,6 +57,7 @@
   overflow-y: auto;
   background-color: #fff;
   overflow-x: hidden;
+  padding: 8px 0px 0px 60px;
 }
 
 .post-title {
@@ -104,7 +105,7 @@
 }
 
 .nside {
-  color: #fb8d08;
+  color: #fb8d08 !important;
 }
 
 .nnav {
@@ -123,4 +124,13 @@
   color:#000000;
 }
 
+#navlist {
+  color:#454545;
+}
+
+#navlist:hover, #navlist:focus{
+  color:#000000;
+}
+
+
 


[45/45] incubator-kylin git commit: Merge tag 'kylin-1.1-incubating'

Posted by sh...@apache.org.
Merge tag 'kylin-1.1-incubating'

[maven-release-plugin] copy for tag kylin-1.1-incubating


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

Branch: refs/heads/master
Commit: 9c9fdfa95d8c226f391fe687c208d0a7e1ec089f
Parents: 43f1197 bac74fb
Author: shaofengshi <sh...@apache.org>
Authored: Mon Oct 26 17:06:17 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Oct 26 17:06:17 2015 +0800

----------------------------------------------------------------------
 KEYS                                            |   58 +
 atopcalcite/pom.xml                             |    2 +-
 .../adapter/enumerable/EnumerableJoin.java      |  146 --
 .../apache/calcite/runtime/SqlFunctions.java    | 1315 ------------------
 .../calcite/sql2rel/SqlToRelConverter.java      |  253 +---
 bin/kylin.sh                                    |   59 +-
 common/pom.xml                                  |    2 +-
 .../org/apache/kylin/common/KylinConfig.java    |   42 +-
 .../apache/kylin/common/util/StringUtil.java    |    8 +-
 .../src/main/resources/kylin-log4j.properties   |   28 +
 common/src/main/resources/kylinlog4j.properties |   27 -
 common/src/main/resources/log4j.properties      |   25 -
 conf/kylin.properties                           |    6 +-
 cube/pom.xml                                    |    8 +-
 .../org/apache/kylin/cube/cuboid/Cuboid.java    |  113 +-
 .../org/apache/kylin/cube/cuboid/CuboidCLI.java |    8 +-
 .../org/apache/kylin/cube/model/CubeDesc.java   |   11 +
 dictionary/pom.xml                              |    2 +-
 .../test_case_data/sandbox/kylin.properties     |  277 ++--
 invertedindex/pom.xml                           |   10 +-
 .../invertedindex/index/BitMapContainer.java    |  504 ++++---
 .../index/ColumnValueContainer.java             |   86 +-
 .../index/CompressedValueContainer.java         |  358 ++---
 .../apache/kylin/invertedindex/index/Slice.java |   18 +-
 jdbc/pom.xml                                    |    2 +-
 .../java/org/apache/kylin/jdbc/KylinMeta.java   |    9 +-
 jdbc/src/main/resources/log4j.properties        |   31 -
 job/pom.xml                                     |    6 +-
 .../org/apache/kylin/job/cube/CubingJob.java    |    5 +-
 .../apache/kylin/job/cube/CubingJobBuilder.java |   29 +-
 .../kylin/job/cube/GarbageCollectionStep.java   |   23 +-
 .../kylin/job/BuildCubeWithEngineTest.java      |    6 +-
 .../apache/kylin/job/BuildIIWithEngineTest.java |    4 +-
 metadata/pom.xml                                |    2 +-
 .../metadata/filter/LogicalTupleFilter.java     |   22 +-
 .../metadata/tool/SandboxMetastoreCLI.java      |    4 +-
 monitor/pom.xml                                 |    2 +-
 .../org/apache/kylin/monitor/ConfigUtils.java   |    6 +-
 .../org/apache/kylin/monitor/DebugClient.java   |    2 +-
 .../apache/kylin/monitor/HiveJdbcClient.java    |    2 +-
 monitor/src/main/resources/log4j.properties     |   31 -
 pom.xml                                         |   26 +-
 query/pom.xml                                   |    2 +-
 .../apache/kylin/query/relnode/OLAPJoinRel.java |   24 +-
 .../kylin/query/relnode/OLAPTableScan.java      |   13 +-
 query/src/test/resources/query/sql/query81.sql  |   28 +
 server/pom.xml                                  |   13 +-
 .../java/org/apache/kylin/rest/DebugTomcat.java |    7 +-
 .../kylin/rest/controller/CubeController.java   |    8 +-
 .../kylin/rest/service/BadQueryDetector.java    |   37 +-
 .../apache/kylin/rest/service/QueryService.java |   21 +-
 .../org/apache/kylin/rest/util/HiveReroute.java |  142 ++
 .../kylin/rest/util/Log4jConfigListener.java    |   46 +
 .../resources/kylin-server-log4j.properties     |   51 +
 server/src/main/resources/log4j.properties      |   55 -
 server/src/main/webapp/WEB-INF/web.xml          |    6 +-
 .../rest/service/BadQueryDetectorTest.java      |    6 +-
 src/main/config/assemblies/source-assembly.xml  |   11 +-
 storage/pom.xml                                 |    9 +-
 .../storage/filter/BitMapFilterEvaluator.java   |  486 +++----
 .../hbase/coprocessor/CoprocessorFilter.java    |    8 +-
 .../hbase/coprocessor/endpoint/IIEndpoint.java  |    6 +-
 .../endpoint/SliceBitMapProvider.java           |    4 +-
 .../coprocessor/endpoint/protobuf/II.proto      |   17 +
 .../coprocessor/observer/ObserverEnabler.java   |    8 +-
 .../filter/BitMapFilterEvaluatorTest.java       |  478 +++----
 webapp/app/js/controllers/cubeEdit.js           |    8 +-
 webapp/app/js/controllers/job.js                |    7 -
 webapp/app/js/controllers/sourceMeta.js         |    6 -
 .../app/partials/tables/source_table_tree.html  |    3 +-
 website/_data/contributors.yml                  |    5 +
 website/_dev/dev_env.md                         |    5 +-
 website/_dev/howto_contribute.md                |   11 +-
 website/_docs/gettingstarted/terminology.md     |    5 +-
 website/_docs/index.md                          |    4 +-
 website/_includes/docs_nav.html                 |    2 +-
 website/_includes/docs_ul.cn.html               |    2 +-
 website/_includes/docs_ul.html                  |    2 +-
 website/_layouts/docs-cn.html                   |    2 +
 website/_layouts/docs.html                      |    5 +-
 website/_posts/blog/2015-09-22-hybrid-model.md  |  136 ++
 website/assets/css/animate.css                  |   17 -
 website/assets/css/docs.css                     |   31 +-
 website/assets/css/styles.css                   |   17 -
 website/assets/fonts/fontawesome-webfont.svg    |   17 -
 website/assets/fonts/helveticaneue-webfont.css  |   17 -
 website/assets/fonts/helveticaneue-webfont.svg  |   17 -
 website/assets/fonts/helveticaneue_lt.css       |   17 -
 website/assets/fonts/helveticaneue_lt.svg       |   17 -
 website/assets/fonts/opensans-light-webfont.svg |   17 -
 .../assets/fonts/opensans-regular-webfont.svg   |   17 -
 website/community/poweredby.md                  |   18 +
 website/css/main.scss                           |   17 -
 website/images/blog/hybrid-model.png            |  Bin 0 -> 118183 bytes
 94 files changed, 2193 insertions(+), 3293 deletions(-)
----------------------------------------------------------------------



[41/45] incubator-kylin git commit: KYLIN-999 exluce .log, .jar and dist/ from the src release

Posted by sh...@apache.org.
KYLIN-999 exluce .log, .jar and dist/ from the src release


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

Branch: refs/heads/master
Commit: a64759c38f9b4fc6a6897c361e7d7ba353aaf0c2
Parents: eca3698
Author: shaofengshi <sh...@apache.org>
Authored: Thu Oct 8 13:56:15 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Oct 8 13:56:43 2015 +0800

----------------------------------------------------------------------
 src/main/config/assemblies/source-assembly.xml | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a64759c3/src/main/config/assemblies/source-assembly.xml
----------------------------------------------------------------------
diff --git a/src/main/config/assemblies/source-assembly.xml b/src/main/config/assemblies/source-assembly.xml
index 87641de..6f54721 100644
--- a/src/main/config/assemblies/source-assembly.xml
+++ b/src/main/config/assemblies/source-assembly.xml
@@ -58,6 +58,8 @@ limitations under the License.
                 </exclude>
                 <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iml]
                 </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.log]
+                </exclude>
                 <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.settings(/.*)?]
                 </exclude>
                 <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.externalToolBuilders(/.*)?]
@@ -88,6 +90,13 @@ limitations under the License.
                 <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?release\.properties]
                 </exclude>
 
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?dist(/.*)?]
+                </exclude>
+
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?build(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?lib(/.*)?]
+                </exclude>
             </excludes>
         </fileSet>
         <!-- LICENSE, NOTICE, DEPENDENCIES, git.properties, etc. calculated at build time -->


[30/45] incubator-kylin git commit: KYLIN-1059 Remove httpclient dependency in job pom to avoid the old commons-codec it derives

Posted by sh...@apache.org.
KYLIN-1059 Remove httpclient dependency in job pom to avoid the old commons-codec it derives


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

Branch: refs/heads/master
Commit: 18610e5a35381566de6be75d6f6196b716a6d2a2
Parents: 0711911
Author: Yang Li <li...@apache.org>
Authored: Tue Oct 6 15:20:25 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Tue Oct 6 15:20:25 2015 +0800

----------------------------------------------------------------------
 job/pom.xml | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/18610e5a/job/pom.xml
----------------------------------------------------------------------
diff --git a/job/pom.xml b/job/pom.xml
index 6df92ba..ba73760 100644
--- a/job/pom.xml
+++ b/job/pom.xml
@@ -81,10 +81,6 @@
             <artifactId>jackson-databind</artifactId>
         </dependency>
         <dependency>
-            <groupId>commons-httpclient</groupId>
-            <artifactId>commons-httpclient</artifactId>
-        </dependency>
-        <dependency>
             <groupId>com.google.guava</groupId>
             <artifactId>guava</artifactId>
         </dependency>


[02/45] incubator-kylin git commit: KYLIN-1035, Validate [Project] before create Cube on UI

Posted by sh...@apache.org.
KYLIN-1035, Validate [Project] before create Cube on UI


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

Branch: refs/heads/master
Commit: df15cad19918086114791546b44fe37a9868ff79
Parents: 607aeb4
Author: jiazhong <ji...@ebay.com>
Authored: Mon Sep 21 11:16:07 2015 +0800
Committer: jiazhong <ji...@ebay.com>
Committed: Mon Sep 21 11:16:46 2015 +0800

----------------------------------------------------------------------
 webapp/app/js/controllers/cubeEdit.js | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/df15cad1/webapp/app/js/controllers/cubeEdit.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeEdit.js b/webapp/app/js/controllers/cubeEdit.js
index 16a9203..2dade99 100755
--- a/webapp/app/js/controllers/cubeEdit.js
+++ b/webapp/app/js/controllers/cubeEdit.js
@@ -19,12 +19,18 @@
 'use strict';
 
 
-KylinApp.controller('CubeEditCtrl', function ($scope, $q, $routeParams, $location, $templateCache, $interpolate, MessageService, TableService, CubeDescService, CubeService, loadingRequest, SweetAlert, $log, cubeConfig, CubeDescModel, ModelService, MetaModel, TableModel) {
+KylinApp.controller('CubeEditCtrl', function ($scope, $q, $routeParams, $location, $templateCache, $interpolate, MessageService, TableService, CubeDescService, CubeService, loadingRequest, SweetAlert, $log, cubeConfig, CubeDescModel, ModelService, MetaModel, TableModel,ProjectModel) {
   $scope.cubeConfig = cubeConfig;
   //add or edit ?
   var absUrl = $location.absUrl();
   $scope.cubeMode = absUrl.indexOf("/cubes/add") != -1 ? 'addNewCube' : absUrl.indexOf("/cubes/edit") != -1 ? 'editExistCube' : 'default';
 
+  //validate project before create
+  if($scope.cubeMode=="addNewCube"&&!ProjectModel.getSelectedProject()){
+    SweetAlert.swal('Oops...', 'Please select your project first.', 'warning');
+    $location.path("/cubes");
+    return;
+  }
 
   $scope.getColumnsByTable = function (tableName) {
     var temp = [];


[26/45] incubator-kylin git commit: KYLIN-457 Change to kylin-log4j.properties

Posted by sh...@apache.org.
KYLIN-457 Change to kylin-log4j.properties


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

Branch: refs/heads/master
Commit: f6f67e79991cfa6a09dec6de0cd3ad14f20bdb33
Parents: 7378115
Author: Yang Li <li...@apache.org>
Authored: Sun Oct 4 19:43:23 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sun Oct 4 19:43:23 2015 +0800

----------------------------------------------------------------------
 bin/kylin.sh                                    |  3 +-
 .../src/main/resources/kylin-log4j.properties   | 27 ++++++++++
 common/src/main/resources/kylinlog4j.properties | 27 ----------
 common/src/main/resources/log4j.properties      | 25 ---------
 jdbc/src/main/resources/log4j.properties        | 31 -----------
 monitor/src/main/resources/log4j.properties     | 31 -----------
 .../java/org/apache/kylin/rest/DebugTomcat.java |  2 +
 .../src/main/resources/kylin-log4j.properties   | 55 ++++++++++++++++++++
 server/src/main/resources/log4j.properties      | 55 --------------------
 server/src/main/webapp/WEB-INF/web.xml          |  2 +-
 10 files changed, 87 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f6f67e79/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/bin/kylin.sh b/bin/kylin.sh
index 1334ed1..c11ddf9 100644
--- a/bin/kylin.sh
+++ b/bin/kylin.sh
@@ -63,6 +63,7 @@ then
     hbase ${KYLIN_EXTRA_START_OPTS} \
     -Djava.util.logging.config.file=${tomcat_root}/conf/logging.properties \
     -Djava.util.logging.manager=org.apache.juli.ClassLoaderLogManager \
+    -Dlog4j.configuration=kylin-log4j.properties \
     -Dorg.apache.tomcat.util.buf.UDecoder.ALLOW_ENCODED_SLASH=true \
     -Dorg.apache.catalina.connector.CoyoteAdapter.ALLOW_BACKSLASH=true \
     -Djava.endorsed.dirs=${tomcat_root}/endorsed  \
@@ -115,7 +116,7 @@ then
 
     export HBASE_CLASSPATH=${KYLIN_HOME}/lib/*:$hive_dependency:${HBASE_CLASSPATH}
 
-    exec hbase "$@"
+    exec hbase -Dlog4j.configuration=kylin-log4j.properties "$@"
 
 else
     echo "usage: kylin.sh start or kylin.sh stop"

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f6f67e79/common/src/main/resources/kylin-log4j.properties
----------------------------------------------------------------------
diff --git a/common/src/main/resources/kylin-log4j.properties b/common/src/main/resources/kylin-log4j.properties
new file mode 100644
index 0000000..d499713
--- /dev/null
+++ b/common/src/main/resources/kylin-log4j.properties
@@ -0,0 +1,27 @@
+#
+# 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.
+#
+
+# enable this by -Dlog4j.configuration=kylin-log4j.properties
+
+log4j.rootLogger=INFO,stdout
+
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=L4J [%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%c] - %m%n
+
+#log4j.logger.org.apache.hadoop=ERROR
+log4j.logger.org.apache.kylin=DEBUG

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f6f67e79/common/src/main/resources/kylinlog4j.properties
----------------------------------------------------------------------
diff --git a/common/src/main/resources/kylinlog4j.properties b/common/src/main/resources/kylinlog4j.properties
deleted file mode 100644
index 564f15d..0000000
--- a/common/src/main/resources/kylinlog4j.properties
+++ /dev/null
@@ -1,27 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-# use this when conflict with hbase, enable this by -Dlog4j.configuration=kylinlog4j.properties
-
-log4j.rootLogger=INFO,stdout
-
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=L4J [%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%c] - %m%n
-
-#log4j.logger.org.apache.hadoop=ERROR
-log4j.logger.org.apache.kylin=DEBUG

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f6f67e79/common/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/common/src/main/resources/log4j.properties b/common/src/main/resources/log4j.properties
deleted file mode 100644
index 8c8b685..0000000
--- a/common/src/main/resources/log4j.properties
+++ /dev/null
@@ -1,25 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-log4j.rootLogger=INFO,stdout
-
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=L4J [%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%c] - %m%n
-
-#log4j.logger.org.apache.hadoop=ERROR
-log4j.logger.org.apache.kylin=DEBUG

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f6f67e79/jdbc/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/jdbc/src/main/resources/log4j.properties b/jdbc/src/main/resources/log4j.properties
deleted file mode 100644
index 7d5d9a3..0000000
--- a/jdbc/src/main/resources/log4j.properties
+++ /dev/null
@@ -1,31 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-#define appenders
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
-
-log4j.appender.file=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.file.layout=org.apache.log4j.PatternLayout
-log4j.appender.file.File=kylin_jdbc.log
-log4j.appender.file.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
-log4j.appender.file.Append=true
-
-#overall config
-log4j.rootLogger=DEBUG,stdout,file
-#log4j.logger.org.apache.kylin=DEBUG,stdout,file
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f6f67e79/monitor/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/monitor/src/main/resources/log4j.properties b/monitor/src/main/resources/log4j.properties
deleted file mode 100644
index 6dec581..0000000
--- a/monitor/src/main/resources/log4j.properties
+++ /dev/null
@@ -1,31 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-log4j.rootLogger=INFO,stdout
-
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=L4J [%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%c] - %m%n
-
-log4j.appender.monitor=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.monitor.layout=org.apache.log4j.PatternLayout
-log4j.appender.monitor.File=${CATALINA_HOME}logs/kylin_monitor.log
-log4j.appender.monitor.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
-log4j.appender.monitor.Append=true
-
-#log4j.logger.org.apache.hadoop=ERROR
-log4j.logger.org.apache.kylin=DEBUG,monitor

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f6f67e79/server/src/main/java/org/apache/kylin/rest/DebugTomcat.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/DebugTomcat.java b/server/src/main/java/org/apache/kylin/rest/DebugTomcat.java
index 8efbae8..fd072ff 100644
--- a/server/src/main/java/org/apache/kylin/rest/DebugTomcat.java
+++ b/server/src/main/java/org/apache/kylin/rest/DebugTomcat.java
@@ -35,6 +35,8 @@ public class DebugTomcat {
 
     public static void setupDebugEnv() {
         try {
+            System.setProperty("log4j.configuration", "kylin-log4j.properties");
+            
             // test_case_data/sandbox/ contains HDP 2.2 site xmls which is dev sandbox
             ClasspathUtil.addClasspath(new File("../examples/test_case_data/sandbox").getAbsolutePath());
             System.setProperty(KylinConfig.KYLIN_CONF, "../examples/test_case_data/sandbox");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f6f67e79/server/src/main/resources/kylin-log4j.properties
----------------------------------------------------------------------
diff --git a/server/src/main/resources/kylin-log4j.properties b/server/src/main/resources/kylin-log4j.properties
new file mode 100644
index 0000000..d7dc3bf
--- /dev/null
+++ b/server/src/main/resources/kylin-log4j.properties
@@ -0,0 +1,55 @@
+#
+# 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.
+#
+
+#define appenders
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
+
+log4j.appender.file=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.File=${catalina.home}/logs/kylin.log
+log4j.appender.file.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
+log4j.appender.file.Append=true
+
+log4j.appender.query=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.query.layout=org.apache.log4j.PatternLayout
+log4j.appender.query.File=${catalina.home}/logs/kylin_query.log
+log4j.appender.query.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
+log4j.appender.query.Append=true
+
+log4j.appender.job=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.job.layout=org.apache.log4j.PatternLayout
+log4j.appender.job.File=${catalina.home}/logs/kylin_job.log
+log4j.appender.job.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
+log4j.appender.job.Append=true
+
+#overall config
+log4j.rootLogger=INFO,stdout,file
+log4j.logger.org.apache.kylin=DEBUG
+log4j.logger.org.springframework=WARN
+
+#query config
+log4j.logger.org.apache.kylin.rest.controller.QueryController=DEBUG, query
+log4j.logger.org.apache.kylin.rest.service.QueryService=DEBUG, query
+log4j.logger.org.apache.kylin.query=DEBUG, query
+log4j.logger.org.apache.kylin.storage=DEBUG, query
+
+#job config
+log4j.logger.org.apache.kylin.rest.controller.JobController=DEBUG, job
+log4j.logger.org.apache.kylin.rest.service.JobService=DEBUG, job
+log4j.logger.org.apache.kylin.job=DEBUG, job

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f6f67e79/server/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/server/src/main/resources/log4j.properties b/server/src/main/resources/log4j.properties
deleted file mode 100644
index d7dc3bf..0000000
--- a/server/src/main/resources/log4j.properties
+++ /dev/null
@@ -1,55 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-#define appenders
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
-
-log4j.appender.file=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.file.layout=org.apache.log4j.PatternLayout
-log4j.appender.file.File=${catalina.home}/logs/kylin.log
-log4j.appender.file.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
-log4j.appender.file.Append=true
-
-log4j.appender.query=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.query.layout=org.apache.log4j.PatternLayout
-log4j.appender.query.File=${catalina.home}/logs/kylin_query.log
-log4j.appender.query.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
-log4j.appender.query.Append=true
-
-log4j.appender.job=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.job.layout=org.apache.log4j.PatternLayout
-log4j.appender.job.File=${catalina.home}/logs/kylin_job.log
-log4j.appender.job.layout.ConversionPattern=[%t]:[%d{yyyy-MM-dd HH:mm:ss,SSS}][%p][%l] - %m%n
-log4j.appender.job.Append=true
-
-#overall config
-log4j.rootLogger=INFO,stdout,file
-log4j.logger.org.apache.kylin=DEBUG
-log4j.logger.org.springframework=WARN
-
-#query config
-log4j.logger.org.apache.kylin.rest.controller.QueryController=DEBUG, query
-log4j.logger.org.apache.kylin.rest.service.QueryService=DEBUG, query
-log4j.logger.org.apache.kylin.query=DEBUG, query
-log4j.logger.org.apache.kylin.storage=DEBUG, query
-
-#job config
-log4j.logger.org.apache.kylin.rest.controller.JobController=DEBUG, job
-log4j.logger.org.apache.kylin.rest.service.JobService=DEBUG, job
-log4j.logger.org.apache.kylin.job=DEBUG, job

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f6f67e79/server/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/server/src/main/webapp/WEB-INF/web.xml b/server/src/main/webapp/WEB-INF/web.xml
index 1573c07..ca5f6d9 100644
--- a/server/src/main/webapp/WEB-INF/web.xml
+++ b/server/src/main/webapp/WEB-INF/web.xml
@@ -32,7 +32,7 @@ limitations under the License. See accompanying LICENSE file.
 
     <context-param>
         <param-name>log4jConfigLocation</param-name>
-        <param-value>classpath:log4j.properties</param-value>
+        <param-value>classpath:kylin-log4j.properties</param-value>
     </context-param>
     <context-param>
         <param-name>contextConfigLocation</param-name>


[20/45] incubator-kylin git commit: KYLIN-1025 Save cube change is very slow

Posted by sh...@apache.org.
KYLIN-1025 Save cube change is very slow

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

Branch: refs/heads/master
Commit: 67322634a767dd7efcbcf3ea5fb879e52c9ce559
Parents: 20f6eaf
Author: shaofengshi <sh...@apache.org>
Authored: Tue Sep 29 13:55:51 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 29 13:55:51 2015 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/kylin/cube/cuboid/CuboidCLI.java    | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/67322634/cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidCLI.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidCLI.java b/cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidCLI.java
index 54320e1..8d53000 100644
--- a/cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidCLI.java
+++ b/cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidCLI.java
@@ -65,14 +65,16 @@ public class CuboidCLI {
             }
         }
 
+        /** disable this due to poor performance when dimension number is big
         TreeSet<Long> enumCuboids = enumCalcCuboidCount(cube);
         if (enumCuboids.equals(cuboidSet) == false) {
             throw new IllegalStateException("Expected cuboid set " + enumCuboids + "; but actual cuboid set " + cuboidSet);
         }
-
+         **/
+        
         int mathCount = mathCalcCuboidCount(cube);
-        if (mathCount != enumCuboids.size()) {
-            throw new IllegalStateException("Math cuboid count " + mathCount + ", but actual cuboid count " + enumCuboids.size());
+        if (mathCount != cuboidSet.size()) {
+            throw new IllegalStateException("Math cuboid count " + mathCount + ", but actual cuboid count " + cuboidSet.size());
         }
 
         return mathCount;


[04/45] incubator-kylin git commit: KYLIN-1037 Remove hardcoded "hdp.version" from regression tests

Posted by sh...@apache.org.
KYLIN-1037 Remove hardcoded "hdp.version" from regression tests


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

Branch: refs/heads/master
Commit: ac10f5dc44f0a3478a9a0e028c49255c11362d11
Parents: 66ae35a
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 21 17:20:39 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 21 17:21:03 2015 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/job/BuildCubeWithEngineTest.java     | 6 ++++--
 .../test/java/org/apache/kylin/job/BuildIIWithEngineTest.java  | 4 +++-
 .../org/apache/kylin/metadata/tool/SandboxMetastoreCLI.java    | 4 +++-
 pom.xml                                                        | 4 ----
 server/src/main/java/org/apache/kylin/rest/DebugTomcat.java    | 5 +++--
 5 files changed, 13 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ac10f5dc/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java b/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
index 28c9061..e8162fe 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
@@ -86,8 +86,10 @@ public class BuildCubeWithEngineTest {
     public static void beforeClass() throws Exception {
         logger.info("Adding to classpath: " + new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
         ClassUtil.addClasspath(new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
-        System.setProperty(KylinConfig.KYLIN_CONF, "../examples/test_case_data/sandbox");
-        System.setProperty("hdp.version", "2.2.4.2-2"); // mapred-site.xml ref this
+        System.setProperty(KylinConfig.KYLIN_CONF, "../examples/test_case_data/sandbox"); 
+        if (System.getProperty("hdp.version") == null) {
+            throw new RuntimeException("No hdp.version set; Please set hdp.version in your jvm option, for example: -Dhdp.version=2.2.4.2-2");
+        }
     }
 
     @Before

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ac10f5dc/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java b/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
index bcacaa6..4d039cf 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
@@ -94,7 +94,9 @@ public class BuildIIWithEngineTest {
     public static void beforeClass() throws Exception {
         logger.info("Adding to classpath: " + new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
         ClassUtil.addClasspath(new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
-        System.setProperty("hdp.version", "2.2.4.2-2"); // mapred-site.xml ref this
+        if (System.getProperty("hdp.version") == null) {
+            throw new RuntimeException("No hdp.version set; Please set hdp.version in your jvm option, for example: -Dhdp.version=2.2.4.2-2");
+        }
     }
 
     @Before

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ac10f5dc/metadata/src/test/java/org/apache/kylin/metadata/tool/SandboxMetastoreCLI.java
----------------------------------------------------------------------
diff --git a/metadata/src/test/java/org/apache/kylin/metadata/tool/SandboxMetastoreCLI.java b/metadata/src/test/java/org/apache/kylin/metadata/tool/SandboxMetastoreCLI.java
index b0cbe2e..5ecedad 100644
--- a/metadata/src/test/java/org/apache/kylin/metadata/tool/SandboxMetastoreCLI.java
+++ b/metadata/src/test/java/org/apache/kylin/metadata/tool/SandboxMetastoreCLI.java
@@ -42,7 +42,9 @@ public class SandboxMetastoreCLI {
         logger.info("Adding to classpath: " + new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
         ClassUtil.addClasspath(new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
         System.setProperty(KylinConfig.KYLIN_CONF, "../examples/test_case_data/sandbox");
-        System.setProperty("hdp.version", "2.2.4.2-2"); // mapred-site.xml ref this
+        if (System.getProperty("hdp.version") == null) {
+            throw new RuntimeException("No hdp.version set; Please set hdp.version in your jvm option, for example: -Dhdp.version=2.2.4.2-2");
+        }
 
         if (args.length < 2) {
             printUsage();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ac10f5dc/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c614478..1f82807 100644
--- a/pom.xml
+++ b/pom.xml
@@ -789,10 +789,6 @@
                                     <value>true</value>
                                 </property>
                                 <property>
-                                    <name>hdp.version</name>
-                                    <value>2.2.0.0-2041</value>
-                                </property>
-                                <property>
                                     <name>buildCubeUsingProvidedData</name>
                                     <value>false</value>
                                 </property>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ac10f5dc/server/src/main/java/org/apache/kylin/rest/DebugTomcat.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/DebugTomcat.java b/server/src/main/java/org/apache/kylin/rest/DebugTomcat.java
index 038dc82..8efbae8 100644
--- a/server/src/main/java/org/apache/kylin/rest/DebugTomcat.java
+++ b/server/src/main/java/org/apache/kylin/rest/DebugTomcat.java
@@ -46,8 +46,9 @@ public class DebugTomcat {
             if (System.getProperty("catalina.home") == null)
                 System.setProperty("catalina.home", ".");
 
-            if (System.getProperty("hdp.version") == null)
-                System.setProperty("hdp.version", "2.2.4.2-2"); // mapred-site.xml ref this
+            if (System.getProperty("hdp.version") == null) {
+                throw new RuntimeException("No hdp.version set; Please set hdp.version in your jvm option, for example: -Dhdp.version=2.2.4.2-2");
+            }
 
             // workaround for job submission from win to linux -- https://issues.apache.org/jira/browse/MAPREDUCE-4052
             if (Shell.WINDOWS) {


[10/45] incubator-kylin git commit: KYLIN-792, update config key in kylin.properties for performance monitor moudle

Posted by sh...@apache.org.
KYLIN-792, update config key in kylin.properties for performance monitor moudle


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

Branch: refs/heads/master
Commit: bad57cb91124cf5a325e1d785a4f5de73d53c8b3
Parents: 62751df
Author: jiazhong <ji...@ebay.com>
Authored: Fri Sep 25 15:27:33 2015 +0800
Committer: jiazhong <ji...@ebay.com>
Committed: Fri Sep 25 15:27:33 2015 +0800

----------------------------------------------------------------------
 conf/kylin.properties                                          | 6 +++---
 examples/test_case_data/sandbox/kylin.properties               | 6 +++---
 .../src/main/java/org/apache/kylin/monitor/ConfigUtils.java    | 6 +++---
 .../src/main/java/org/apache/kylin/monitor/DebugClient.java    | 2 +-
 .../src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java | 2 +-
 5 files changed, 11 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bad57cb9/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index 2b8441a..e6f5a68 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -131,11 +131,11 @@ kylin.sandbox=true
 
 ###########################config info for kylin monitor#######################
 # hive jdbc url
-kylin.hive.jdbc.connection.url=
+kylin.monitor.hive.jdbc.connection.url=
 
 #config where to parse query log,split with comma ,will also read $KYLIN_HOME/tomcat/logs/ by default
-ext.log.base.dir = /tmp/kylin_log1,/tmp/kylin_log2
+kylin.monitor.ext.log.base.dir = /tmp/kylin_log1,/tmp/kylin_log2
 
 #will create external hive table to query result csv file
 #will set to kylin_query_log by default if not config here
-query.log.parse.result.table = kylin_query_log
+kylin.monitor.query.log.parse.result.table = kylin_query_log

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bad57cb9/examples/test_case_data/sandbox/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin.properties b/examples/test_case_data/sandbox/kylin.properties
index e4b20fe..bcae574 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -123,14 +123,14 @@ kylin.sandbox=true
 
 ###########################config info for kylin monitor#######################
 # hive jdbc url
-kylin.hive.jdbc.connection.url= jdbc:hive2://sandbox:10000
+kylin.monitor.hive.jdbc.connection.url= jdbc:hive2://sandbox:10000
 
 #config where to parse query log,split with comma ,will also read $KYLIN_HOME/tomcat/logs/ by default
-ext.log.base.dir = /tmp/kylin_log1,/tmp/kylin_log2
+kylin.monitor.ext.log.base.dir = /tmp/kylin_log1,/tmp/kylin_log2
 
 #will create external hive table to query result csv file
 #will set to kylin_query_log by default if not config here
-query.log.parse.result.table = kylin_query_log
+kylin.monitor.query.log.parse.result.table = kylin_query_log
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bad57cb9/monitor/src/main/java/org/apache/kylin/monitor/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/ConfigUtils.java b/monitor/src/main/java/org/apache/kylin/monitor/ConfigUtils.java
index 7cff4d7..6d71513 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/ConfigUtils.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/ConfigUtils.java
@@ -51,7 +51,7 @@ public class ConfigUtils {
     private ConfigUtils() {
     }
 
-    public static final String KYLIN_EXT_LOG_BASE_DIR = "ext.log.base.dir";
+    public static final String KYLIN_EXT_LOG_BASE_DIR = "kylin.monitor.ext.log.base.dir";
     public static final String KYLIN_METADATA_URL = "kylin.metadata.url";
 
     public static final String KYLIN_HOME = "KYLIN_HOME";
@@ -62,7 +62,7 @@ public class ConfigUtils {
     public static final String KYLIN_HDFS_WORKING_DIR = "kylin.hdfs.working.dir";
 
     public static final String KYLIN_MONITOR_CONF_PROP_FILE = "kylin.properties";
-    public static final String QUERY_LOG_PARSE_RESULT_TABLE = "query.log.parse.result.table";
+    public static final String QUERY_LOG_PARSE_RESULT_TABLE = "kylin.monitor.query.log.parse.result.table";
     public static final String DEFAULT_QUERY_LOG_PARSE_RESULT_TABLE = "kylin_query_log";
 
     public static final String HIVE_JDBC_CON_USERNAME = "kylin.hive.jdbc.connection.username";
@@ -72,7 +72,7 @@ public class ConfigUtils {
 
     public static final String DEPLOY_ENV = "deploy.env";
 
-    public static final String HIVE_JDBC_CON_URL = "kylin.hive.jdbc.connection.url";
+    public static final String HIVE_JDBC_CON_URL = "kylin.monitor.hive.jdbc.connection.url";
 
     public void loadMonitorParam() throws IOException {
         Properties props = new Properties();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bad57cb9/monitor/src/main/java/org/apache/kylin/monitor/DebugClient.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/DebugClient.java b/monitor/src/main/java/org/apache/kylin/monitor/DebugClient.java
index 3a683da..67bd630 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/DebugClient.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/DebugClient.java
@@ -39,7 +39,7 @@ public class DebugClient {
         // test_case_data/sandbox/ contains HDP 2.2 site xmls which is dev sandbox
         ConfigUtils.addClasspath(new File("../examples/test_case_data/sandbox").getAbsolutePath());
 
-        //set log base dir ,will also get from $KYLIN_HOME/tomcat/logs and config [ext.log.base.dir] in kylin.properties
+        //set log base dir ,will also get from $KYLIN_HOME/tomcat/logs and config [kylin.monitor.ext.log.base.dir] in kylin.properties
         System.setProperty(ConfigUtils.KYLIN_LOG_CONF_HOME, "../server/logs");
 
         //get kylin.properties ,if not exist will get from $KYLIN_HOME/conf/

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bad57cb9/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java b/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
index ffe3cc0..0216abf 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
@@ -77,7 +77,7 @@ public class HiveJdbcClient {
             monitorConfig.loadMonitorParam();
             QUERY_LOG_TABLE_NAME = monitorConfig.getQueryLogResultTable();
             if (StringUtils.isEmpty(QUERY_LOG_TABLE_NAME)) {
-                logger.error("table name not defined ,please set param [query.log.parse.result.table] in kylin.properties");
+                logger.error("table name not defined ,please set param [kylin.monitor.query.log.parse.result.table] in kylin.properties");
             }
 
         } catch (Exception e) {


[34/45] incubator-kylin git commit: add powered by page

Posted by sh...@apache.org.
add powered by page


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

Branch: refs/heads/master
Commit: 290f5cef4b2524e27d51562ec478bfefa187d30e
Parents: 1c1a24c
Author: lukehan <lu...@apache.org>
Authored: Wed Oct 7 23:53:58 2015 +0800
Committer: lukehan <lu...@apache.org>
Committed: Wed Oct 7 23:53:58 2015 +0800

----------------------------------------------------------------------
 website/_data/contributors.yml |  5 +++++
 website/community/poweredby.md | 18 ++++++++++++++++++
 2 files changed, 23 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/290f5cef/website/_data/contributors.yml
----------------------------------------------------------------------
diff --git a/website/_data/contributors.yml b/website/_data/contributors.yml
index 56c5b53..dfc3d9b 100644
--- a/website/_data/contributors.yml
+++ b/website/_data/contributors.yml
@@ -75,6 +75,11 @@
   githubId: xduo
   org: Alipay
   role: PMC
+- name: Ankur Bansal
+  apacheId: abansal
+  githubId: abansal
+  org: eBay
+  role: PMC
 - name: Xu Jiang (蒋旭)
   apacheId: jiangxu
   githubId: jiangxuchina

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/290f5cef/website/community/poweredby.md
----------------------------------------------------------------------
diff --git a/website/community/poweredby.md b/website/community/poweredby.md
new file mode 100644
index 0000000..8312ac1
--- /dev/null
+++ b/website/community/poweredby.md
@@ -0,0 +1,18 @@
+---
+layout: default
+title: PoweredBy
+permalink: /community/poweredby.html
+---		
+
+## Powered By Apache Kylin
+
+__How to be listed here?__
+_Send a quick description of your organization and usage to the [mailing list](mailto:user@kylin.incubator.apache.org) or to [@apachekylin](https://twitter.com/apachekylin) or [@lukehq](https://twitter.com/lukehq) on twitter and we'll add you._
+
+__Companies__
+
+* [eBay](http://www.ebay.com), Kylin was developed inside eBay for Big Data Analytics on Hadoop and then contributed to open source world in Oct 2014, accepted as Apache Incubator project in Nov 2014. Apache Kylin is used at eBay for OLAP on Hadoop. This powers various data products including Behavior Analytics, Traffic Reporting, Account Manager Application and Streaming Dashboard.
+* [MiningLAMP](http://www.mininglamp.com/), Kylin is part of MiningLAMP's Big Data Analytics product offering OLAP capability 
+* [Meituan](http://www.meituan.com)
+* TBD...
+


[25/45] incubator-kylin git commit: KYLIN-999 remove incorrect copyright from website/

Posted by sh...@apache.org.
KYLIN-999 remove incorrect copyright from website/


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

Branch: refs/heads/master
Commit: 7378115eb2e3923b2244df1c7be59c0b7ca87ae6
Parents: b22ca0d
Author: shaofengshi <sh...@apache.org>
Authored: Wed Sep 30 15:17:05 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Wed Sep 30 15:33:32 2015 +0800

----------------------------------------------------------------------
 website/assets/css/animate.css                    | 17 -----------------
 website/assets/css/docs.css                       | 17 -----------------
 website/assets/css/styles.css                     | 17 -----------------
 website/assets/fonts/fontawesome-webfont.svg      | 17 -----------------
 website/assets/fonts/helveticaneue-webfont.css    | 17 -----------------
 website/assets/fonts/helveticaneue-webfont.svg    | 17 -----------------
 website/assets/fonts/helveticaneue_lt.css         | 17 -----------------
 website/assets/fonts/helveticaneue_lt.svg         | 17 -----------------
 website/assets/fonts/opensans-light-webfont.svg   | 17 -----------------
 website/assets/fonts/opensans-regular-webfont.svg | 17 -----------------
 website/css/main.scss                             | 17 -----------------
 11 files changed, 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7378115e/website/assets/css/animate.css
----------------------------------------------------------------------
diff --git a/website/assets/css/animate.css b/website/assets/css/animate.css
index b090c1e..d71da17 100644
--- a/website/assets/css/animate.css
+++ b/website/assets/css/animate.css
@@ -1,20 +1,3 @@
-/*
- * 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.
- */
-
 @charset "UTF-8";
 
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7378115e/website/assets/css/docs.css
----------------------------------------------------------------------
diff --git a/website/assets/css/docs.css b/website/assets/css/docs.css
index e5dbfc9..8e5d484 100755
--- a/website/assets/css/docs.css
+++ b/website/assets/css/docs.css
@@ -1,20 +1,3 @@
-/*
- * 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.
- */
-
 .aside {
   height: 100%;
   background-color:#fff;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7378115e/website/assets/css/styles.css
----------------------------------------------------------------------
diff --git a/website/assets/css/styles.css b/website/assets/css/styles.css
index 2648206..c1c935f 100644
--- a/website/assets/css/styles.css
+++ b/website/assets/css/styles.css
@@ -1,20 +1,3 @@
-/*
- * 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.
- */
-
 @font-face {
 	font-family: 'HelveticaNeueLight';
 	src: url('../fonts/helveticaneue_lt.eot');

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7378115e/website/assets/fonts/fontawesome-webfont.svg
----------------------------------------------------------------------
diff --git a/website/assets/fonts/fontawesome-webfont.svg b/website/assets/fonts/fontawesome-webfont.svg
index cff2549..d907b25 100644
--- a/website/assets/fonts/fontawesome-webfont.svg
+++ b/website/assets/fonts/fontawesome-webfont.svg
@@ -1,21 +1,4 @@
 <?xml version="1.0" standalone="no"?>
-<!--
-* 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.
--->
 <!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
 <svg xmlns="http://www.w3.org/2000/svg">
 <metadata></metadata>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7378115e/website/assets/fonts/helveticaneue-webfont.css
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue-webfont.css b/website/assets/fonts/helveticaneue-webfont.css
index 8c91d0d..6a42ad4 100644
--- a/website/assets/fonts/helveticaneue-webfont.css
+++ b/website/assets/fonts/helveticaneue-webfont.css
@@ -1,20 +1,3 @@
-/*
- * 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.
- */
-
 @font-face {
     font-family: 'HelveticaNeueRegular';
     src: url('helveticaneue-webfont.eot');

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7378115e/website/assets/fonts/helveticaneue-webfont.svg
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue-webfont.svg b/website/assets/fonts/helveticaneue-webfont.svg
index 0feefca..d6476d6 100644
--- a/website/assets/fonts/helveticaneue-webfont.svg
+++ b/website/assets/fonts/helveticaneue-webfont.svg
@@ -1,21 +1,4 @@
 <?xml version="1.0" standalone="no"?>
-<!--
-* 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.
--->
 <!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
 <svg>
 <metadata>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7378115e/website/assets/fonts/helveticaneue_lt.css
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue_lt.css b/website/assets/fonts/helveticaneue_lt.css
index f02d428..cb4cb73 100644
--- a/website/assets/fonts/helveticaneue_lt.css
+++ b/website/assets/fonts/helveticaneue_lt.css
@@ -1,20 +1,3 @@
-/*
- * 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.
- */
-
 @font-face {
     font-family: 'HelveticaNeueLight';
     src: url('helveticaneue_lt.eot');

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7378115e/website/assets/fonts/helveticaneue_lt.svg
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue_lt.svg b/website/assets/fonts/helveticaneue_lt.svg
index 2db0e13..112404c 100644
--- a/website/assets/fonts/helveticaneue_lt.svg
+++ b/website/assets/fonts/helveticaneue_lt.svg
@@ -1,21 +1,4 @@
 <?xml version="1.0" standalone="no"?>
-<!--
-* 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.
--->
 <!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
 <svg>
 <metadata>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7378115e/website/assets/fonts/opensans-light-webfont.svg
----------------------------------------------------------------------
diff --git a/website/assets/fonts/opensans-light-webfont.svg b/website/assets/fonts/opensans-light-webfont.svg
index 7e19254..a36a5b7 100644
--- a/website/assets/fonts/opensans-light-webfont.svg
+++ b/website/assets/fonts/opensans-light-webfont.svg
@@ -1,21 +1,4 @@
 <?xml version="1.0" standalone="no"?>
-<!--
-* 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.
--->
 <!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
 <svg xmlns="http://www.w3.org/2000/svg">
 <metadata></metadata>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7378115e/website/assets/fonts/opensans-regular-webfont.svg
----------------------------------------------------------------------
diff --git a/website/assets/fonts/opensans-regular-webfont.svg b/website/assets/fonts/opensans-regular-webfont.svg
index 0b97060..a169e01 100644
--- a/website/assets/fonts/opensans-regular-webfont.svg
+++ b/website/assets/fonts/opensans-regular-webfont.svg
@@ -1,21 +1,4 @@
 <?xml version="1.0" standalone="no"?>
-<!--
-* 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.
--->
 <!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
 <svg xmlns="http://www.w3.org/2000/svg">
 <metadata></metadata>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7378115e/website/css/main.scss
----------------------------------------------------------------------
diff --git a/website/css/main.scss b/website/css/main.scss
index f077752..beee4e3 100755
--- a/website/css/main.scss
+++ b/website/css/main.scss
@@ -1,20 +1,3 @@
-/*
- * 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.
- */
-
 ---
 # Only the main Sass file needs front matter (the dashes are enough)
 ---


[03/45] incubator-kylin git commit: KYLIN-1036, Code Clean

Posted by sh...@apache.org.
KYLIN-1036, Code Clean


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

Branch: refs/heads/master
Commit: 66ae35a17c407f3ca5cd9026279c40c6c8ffbd28
Parents: df15cad
Author: jiazhong <ji...@ebay.com>
Authored: Mon Sep 21 14:28:08 2015 +0800
Committer: jiazhong <ji...@ebay.com>
Committed: Mon Sep 21 14:28:08 2015 +0800

----------------------------------------------------------------------
 webapp/app/js/controllers/job.js                  | 7 -------
 webapp/app/js/controllers/sourceMeta.js           | 6 ------
 webapp/app/partials/tables/source_table_tree.html | 3 +--
 3 files changed, 1 insertion(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/66ae35a1/webapp/app/js/controllers/job.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/job.js b/webapp/app/js/controllers/job.js
index 79913e1..b1360aa 100644
--- a/webapp/app/js/controllers/job.js
+++ b/webapp/app/js/controllers/job.js
@@ -25,7 +25,6 @@ KylinApp
     JobList.removeAll();
     $scope.jobConfig = jobConfig;
     $scope.cubeName = null;
-    $scope.projects = [];
     $scope.action = {};
 
     $scope.status = [];
@@ -49,12 +48,6 @@ KylinApp
       projectName: $scope.projectModel.selectedProject
     };
 
-    ProjectService.list({}, function (projects) {
-      angular.forEach(projects, function (project, index) {
-        $scope.projects.push(project.name);
-      });
-    });
-
     $scope.list = function (offset, limit) {
       var defer = $q.defer();
       if (!$scope.projectModel.projects.length) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/66ae35a1/webapp/app/js/controllers/sourceMeta.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/sourceMeta.js b/webapp/app/js/controllers/sourceMeta.js
index 91e771b..d05dc7f 100755
--- a/webapp/app/js/controllers/sourceMeta.js
+++ b/webapp/app/js/controllers/sourceMeta.js
@@ -76,12 +76,6 @@ KylinApp
       }
     };
 
-    $scope.selectedNode = function(obj){
-      if (obj&&obj.uuid) {
-        return $scope.tableModel.selectedSrcTable;
-      }
-    }
-
     $scope.aceSrcTbChanged = function () {
       $scope.tableModel.selectedSrcDb = [];
       $scope.tableModel.selectedSrcTable = {};

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/66ae35a1/webapp/app/partials/tables/source_table_tree.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/tables/source_table_tree.html b/webapp/app/partials/tables/source_table_tree.html
index 1e6e08f..bdbe079 100755
--- a/webapp/app/partials/tables/source_table_tree.html
+++ b/webapp/app/partials/tables/source_table_tree.html
@@ -38,8 +38,7 @@
                      dirSelection="true"
                      tree-model="tableModel.selectedSrcDb"
                      options="tableModel.treeOptions"
-                     on-selection="showSelected(node)"
-                     selected-node="selectedNode(node)">
+                     on-selection="showSelected(node)">
             {{node.name}} {{!!(node.datatype)?'(' + trimType(node.datatype) + ')' : ''}}
         </treecontrol>
     </div>


[43/45] incubator-kylin git commit: KYLIN-1052 fix release name in pom.xml

Posted by sh...@apache.org.
KYLIN-1052 fix release name in pom.xml

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

Branch: refs/heads/master
Commit: 1955a2f9aea7b7f608f0496c00807715ea4246a5
Parents: b545f00
Author: shaofengshi <sh...@apache.org>
Authored: Thu Oct 8 16:30:48 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Oct 8 16:30:48 2015 +0800

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1955a2f9/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index db9ab91..aca1ff1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -837,7 +837,7 @@
                                     <runOnlyAtExecutionRoot>true</runOnlyAtExecutionRoot>
                                     <appendAssemblyId>true</appendAssemblyId>
                                     <descriptor>src/main/config/assemblies/source-assembly.xml</descriptor>
-                                    <finalName>apache-kylin-${project.version}-src</finalName>
+                                    <finalName>apache-kylin-${project.version}</finalName>
                                 </configuration>
                             </execution>
                         </executions>


[18/45] incubator-kylin git commit: KYLIN-1045 Fix exclude in pom.xml for RAT

Posted by sh...@apache.org.
KYLIN-1045 Fix exclude in pom.xml for RAT


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

Branch: refs/heads/master
Commit: 734bf3aa36582ff3bbc6b1bb5ce07d0b4674ba5d
Parents: 9ac01db
Author: sunyerui <su...@gmail.com>
Authored: Mon Sep 28 17:34:18 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Mon Sep 28 17:35:24 2015 +0800

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


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/734bf3aa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ae9714c..56ed278 100644
--- a/pom.xml
+++ b/pom.xml
@@ -621,6 +621,7 @@
                                 <exclude>**/*.snapshot</exclude>
                                 <exclude>**/*.pdf</exclude>
                                 <exclude>**/*.log</exclude>
+                                <exclude>dist/**</exclude>
 
                                 <exclude>**/.checkstyle</exclude>
                                 <!--Job's Test Data-->


[05/45] incubator-kylin git commit: minor, append kylin.log, don't overwrite

Posted by sh...@apache.org.
minor, append kylin.log, don't overwrite


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

Branch: refs/heads/master
Commit: d86829ae67a906080ecefd540806ae9fd501e010
Parents: ac10f5d
Author: Li, Yang <ya...@ebay.com>
Authored: Tue Sep 22 10:10:46 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Tue Sep 22 10:10:46 2015 +0800

----------------------------------------------------------------------
 bin/kylin.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d86829ae/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/bin/kylin.sh b/bin/kylin.sh
index fbaff24..ea84f22 100644
--- a/bin/kylin.sh
+++ b/bin/kylin.sh
@@ -71,7 +71,7 @@ then
     -Dkylin.hive.dependency=${hive_dependency} \
     -Dkylin.hbase.dependency=${hbase_dependency} \
     -Dspring.profiles.active=${spring_profile} \
-    org.apache.hadoop.util.RunJar ${tomcat_root}/bin/bootstrap.jar  org.apache.catalina.startup.Bootstrap start > ${tomcat_root}/logs/kylin.log 2>&1 & echo $! > ${KYLIN_HOME}/pid &
+    org.apache.hadoop.util.RunJar ${tomcat_root}/bin/bootstrap.jar  org.apache.catalina.startup.Bootstrap start >> ${tomcat_root}/logs/kylin.log 2>&1 & echo $! > ${KYLIN_HOME}/pid &
     echo "A new Kylin instance is started by $USER, stop it using \"kylin.sh stop\""
     if [ "$useSandbox" = "true" ]
         then echo "Please visit http://<your_sandbox_ip>:7070/kylin to play with the cubes! (Useranme: ADMIN, Password: KYLIN)"