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

[17/47] incubator-kylin git commit: KYLIN-875 rename modules: core-common, core-cube, core-dictionary, core-cube

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/ITupleIterator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/ITupleIterator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/ITupleIterator.java
new file mode 100644
index 0000000..6bab137
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/ITupleIterator.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.metadata.tuple;
+
+import java.util.Iterator;
+
+/**
+ * @author xjiang
+ *
+ */
+public interface ITupleIterator extends Iterator<ITuple> {
+    ITupleIterator EMPTY_TUPLE_ITERATOR = new ITupleIterator() {
+        @Override
+        public boolean hasNext() {
+            return false;
+        }
+
+        @Override
+        public ITuple next() {
+            return null;
+        }
+
+        @Override
+        public void remove() {
+
+        }
+
+        @Override
+        public void close() {
+        }
+    };
+
+    void close();
+
+
+    /**
+     * if hasNext() returns false because there's no more data, return true
+     * if hasNext() returns false because limits or threshold, return false
+     * if hasNext() returns true, throw IllegalStateException
+     * @return
+     */
+    //public boolean isDrained();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/SimpleTupleIterator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/SimpleTupleIterator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/SimpleTupleIterator.java
new file mode 100644
index 0000000..92006e1
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/SimpleTupleIterator.java
@@ -0,0 +1,35 @@
+package org.apache.kylin.metadata.tuple;
+
+import java.util.Iterator;
+
+/**
+ *
+ */
+public class SimpleTupleIterator implements ITupleIterator {
+
+    private Iterator<ITuple> backend;
+
+    public SimpleTupleIterator(Iterator<ITuple> backend) {
+        this.backend = backend;
+    }
+
+    @Override
+    public boolean hasNext() {
+        return backend.hasNext();
+    }
+
+    @Override
+    public ITuple next() {
+        return backend.next();
+    }
+
+    @Override
+    public void remove() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void close() {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TeeTupleIterator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TeeTupleIterator.java b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TeeTupleIterator.java
new file mode 100644
index 0000000..43b469c
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TeeTupleIterator.java
@@ -0,0 +1,58 @@
+package org.apache.kylin.metadata.tuple;
+
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+/**
+ * Like "tee" command in linux, it effectively duplicates the underlying
+ * ITupleIterator's results
+ */
+public class TeeTupleIterator implements ITupleIterator {
+
+    private static final Logger logger = LoggerFactory.getLogger(TeeTupleIterator.class);
+
+    private ITupleIterator underlying;
+    private List<ITuple> duplicatedData;
+    private List<TeeTupleItrListener> listeners = Lists.newArrayList();
+    private long createTime;
+
+    public TeeTupleIterator(ITupleIterator underlying) {
+        this.underlying = underlying;
+        this.duplicatedData = Lists.newArrayList();
+        this.createTime = System.currentTimeMillis();
+    }
+
+    @Override
+    public void close() {
+        this.underlying.close();
+
+
+        for (TeeTupleItrListener listener : this.listeners) {
+            listener.notify(this.duplicatedData,this.createTime);
+        }
+    }
+
+    @Override
+    public boolean hasNext() {
+        return this.underlying.hasNext();
+    }
+
+    @Override
+    public ITuple next() {
+        ITuple ret = this.underlying.next();
+        duplicatedData.add(ret.makeCopy());
+        return ret;
+    }
+
+    @Override
+    public void remove() {
+        this.underlying.remove();
+    }
+
+    public void addCloseListener(TeeTupleItrListener listener) {
+        this.listeners.add(listener);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TeeTupleItrListener.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TeeTupleItrListener.java b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TeeTupleItrListener.java
new file mode 100644
index 0000000..dfc2625
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/TeeTupleItrListener.java
@@ -0,0 +1,9 @@
+package org.apache.kylin.metadata.tuple;
+
+import java.util.List;
+
+/**
+ */
+public interface TeeTupleItrListener {
+    void notify(List<ITuple> duplicated,long createTime);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/main/java/org/apache/kylin/source/ITableSource.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/source/ITableSource.java b/core-metadata/src/main/java/org/apache/kylin/source/ITableSource.java
new file mode 100644
index 0000000..b2cd3b8
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/source/ITableSource.java
@@ -0,0 +1,28 @@
+/*
+ * 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.source;
+
+import org.apache.kylin.metadata.model.TableDesc;
+
+public interface ITableSource {
+
+    public <I> I adaptToBuildEngine(Class<I> engineInterface);
+    
+    public ReadableTable createReadableTable(TableDesc tableDesc);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/main/java/org/apache/kylin/source/ReadableTable.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/source/ReadableTable.java b/core-metadata/src/main/java/org/apache/kylin/source/ReadableTable.java
new file mode 100644
index 0000000..19c7790
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/source/ReadableTable.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.source;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+
+/**
+ */
+public interface ReadableTable {
+
+    /** Returns a reader to read the table. */
+    public TableReader getReader() throws IOException;
+
+    /** Used to detect table modifications mainly. Return null in case table does not exist. */
+    public TableSignature getSignature() throws IOException;
+
+    public interface TableReader extends Closeable {
+
+        /** Move to the next row, return false if no more record. */
+        public boolean next() throws IOException;
+
+        /** Get the current row. */
+        public String[] getRow();
+        
+    }
+    
+    // ============================================================================
+    
+    @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
+    public class TableSignature {
+
+        @JsonProperty("path")
+        private String path;
+        @JsonProperty("size")
+        private long size;
+        @JsonProperty("last_modified_time")
+        private long lastModifiedTime;
+
+        // for JSON serialization
+        public TableSignature() {
+        }
+
+        public TableSignature(String path, long size, long lastModifiedTime) {
+            super();
+            this.path = path;
+            this.size = size;
+            this.lastModifiedTime = lastModifiedTime;
+        }
+
+        public void setPath(String path) {
+            this.path = path;
+        }
+
+        public void setSize(long size) {
+            this.size = size;
+        }
+
+        public void setLastModifiedTime(long lastModifiedTime) {
+            this.lastModifiedTime = lastModifiedTime;
+        }
+
+        public String getPath() {
+            return path;
+        }
+
+        public long getSize() {
+            return size;
+        }
+
+        public long getLastModifiedTime() {
+            return lastModifiedTime;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + (int) (lastModifiedTime ^ (lastModifiedTime >>> 32));
+            result = prime * result + ((path == null) ? 0 : path.hashCode());
+            result = prime * result + (int) (size ^ (size >>> 32));
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj)
+                return true;
+            if (obj == null)
+                return false;
+            if (getClass() != obj.getClass())
+                return false;
+            TableSignature other = (TableSignature) obj;
+            if (lastModifiedTime != other.lastModifiedTime)
+                return false;
+            if (path == null) {
+                if (other.path != null)
+                    return false;
+            } else if (!path.equals(other.path))
+                return false;
+            if (size != other.size)
+                return false;
+            return true;
+        }
+
+        @Override
+        public String toString() {
+            return "FileSignature [path=" + path + ", size=" + size + ", lastModifiedTime=" + lastModifiedTime + "]";
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/main/java/org/apache/kylin/source/TableSourceFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/source/TableSourceFactory.java b/core-metadata/src/main/java/org/apache/kylin/source/TableSourceFactory.java
new file mode 100644
index 0000000..37ca313
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/source/TableSourceFactory.java
@@ -0,0 +1,40 @@
+/*
+ * 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.source;
+
+import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.metadata.model.IBuildable;
+import org.apache.kylin.metadata.model.TableDesc;
+
+public class TableSourceFactory {
+
+    private static ITableSource dft = (ITableSource) ClassUtil.newInstance("org.apache.kylin.source.hive.HiveTableSource");
+    
+    public static ReadableTable createReadableTable(TableDesc table) {
+        return dft.createReadableTable(table);
+    }
+    
+    public static <T> T createEngineAdapter(IBuildable buildable, Class<T> engineInterface) {
+        return dft.adaptToBuildEngine(engineInterface);
+    }
+    
+    public static <T> T createEngineAdapter(TableDesc tableDesc, Class<T> engineInterface) {
+        return dft.adaptToBuildEngine(engineInterface);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/test/java/org/apache/kylin/metadata/MetadataManagerTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/MetadataManagerTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/MetadataManagerTest.java
new file mode 100644
index 0000000..e8fcad9
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/metadata/MetadataManagerTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.metadata;
+
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.apache.kylin.metadata.MetadataManager.getInstance;
+
+/**
+ */
+public class MetadataManagerTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testListAllTables() throws Exception {
+        List<TableDesc> tables = getInstance(getTestConfig()).listAllTables();
+        Assert.assertNotNull(tables);
+        Assert.assertTrue(tables.size() > 0);
+    }
+
+    @Test
+    public void testFindTableByName() throws Exception {
+        TableDesc table = getInstance(getTestConfig()).getTableDesc("EDW.TEST_CAL_DT");
+        Assert.assertNotNull(table);
+        Assert.assertEquals("EDW.TEST_CAL_DT", table.getIdentity());
+    }
+
+    @Test
+    public void testGetInstance() throws Exception {
+
+        Assert.assertNotNull(getInstance(getTestConfig()));
+        Assert.assertNotNull(getInstance(getTestConfig()).listAllTables());
+        Assert.assertTrue(getInstance(getTestConfig()).listAllTables().size() > 0);
+    }
+
+    @Test
+    public void testDataModel() throws Exception {
+        DataModelDesc modelDesc = getInstance(getTestConfig()).getDataModelDesc("test_kylin_left_join_model_desc");
+
+        Assert.assertTrue(modelDesc.getDimensions().size()>0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/test/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodecTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodecTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodecTest.java
new file mode 100644
index 0000000..bb30c1e
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/metadata/measure/fixedlen/FixedPointLongCodecTest.java
@@ -0,0 +1,49 @@
+package org.apache.kylin.metadata.measure.fixedlen;
+
+import org.apache.kylin.metadata.model.DataType;
+import org.junit.Test;
+
+/**
+ */
+public class FixedPointLongCodecTest {
+
+    @Test
+    public void testEncode1()
+    {
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        long x = codec.getValueIgnoringDecimalPoint("12.12345");
+        org.junit.Assert.assertEquals(1212345,x);
+    }
+
+    @Test
+    public void testEncode2()
+    {
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        long x = codec.getValueIgnoringDecimalPoint("12.1234");
+        org.junit.Assert.assertEquals(1212340,x);
+    }
+
+    @Test
+    public void testEncode3()
+    {
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        long x = codec.getValueIgnoringDecimalPoint("12.123456");
+        org.junit.Assert.assertEquals(1212345,x);
+    }
+
+    @Test
+    public void testEncode4()
+    {
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        long x = codec.getValueIgnoringDecimalPoint("12");
+        org.junit.Assert.assertEquals(1200000,x);
+    }
+
+    @Test
+    public void testDecode1()
+    {
+        FixedPointLongCodec codec = new FixedPointLongCodec(DataType.getInstance("decimal(18,5)"));
+        String x = codec.restoreDecimalPoint(1212345);
+        org.junit.Assert.assertEquals("12.12345",x);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/test/java/org/apache/kylin/metadata/model/TableDescTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/model/TableDescTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/model/TableDescTest.java
new file mode 100644
index 0000000..3a69e5c
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/metadata/model/TableDescTest.java
@@ -0,0 +1,26 @@
+/*
+ * 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.metadata.model;
+
+
+/**
+ */
+public class TableDescTest {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/test/java/org/apache/kylin/metadata/serializer/BigDecimalSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/serializer/BigDecimalSerializerTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/serializer/BigDecimalSerializerTest.java
new file mode 100644
index 0000000..e7b7916
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/metadata/serializer/BigDecimalSerializerTest.java
@@ -0,0 +1,52 @@
+package org.apache.kylin.metadata.serializer;
+
+import org.apache.kylin.metadata.model.DataType;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ */
+public class BigDecimalSerializerTest {
+
+    private static BigDecimalSerializer bigDecimalSerializer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        bigDecimalSerializer = new BigDecimalSerializer(DataType.getInstance("decimal"));
+    }
+
+    @Test
+    public void testNormal() {
+        BigDecimal input = new BigDecimal("1234.1234");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        buffer.mark();
+        bigDecimalSerializer.serialize(input, buffer);
+        buffer.reset();
+        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
+        assertEquals(input, output);
+    }
+
+    @Test
+    public void testScaleOutOfRange() {
+        BigDecimal input = new BigDecimal("1234.1234567890");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        buffer.mark();
+        bigDecimalSerializer.serialize(input, buffer);
+        buffer.reset();
+        BigDecimal output = bigDecimalSerializer.deserialize(buffer);
+        assertEquals(input.setScale(bigDecimalSerializer.type.getScale(), BigDecimal.ROUND_HALF_EVEN), output);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testOutOfPrecision() {
+        BigDecimal input = new BigDecimal("66855344214907231736.4924");
+        ByteBuffer buffer = ByteBuffer.allocate(256);
+        bigDecimalSerializer.serialize(input, buffer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/test/java/org/apache/kylin/metadata/tuple/EmptyTupleIteratorTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/tuple/EmptyTupleIteratorTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/tuple/EmptyTupleIteratorTest.java
new file mode 100644
index 0000000..b3083dc
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/metadata/tuple/EmptyTupleIteratorTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.metadata.tuple;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+
+public class EmptyTupleIteratorTest {
+
+    @Test
+    public void testListAllTables() throws Exception {
+        ITupleIterator it = ITupleIterator.EMPTY_TUPLE_ITERATOR;
+        assertFalse(it.hasNext());
+        assertNull(it.next());
+        it.close(); // for coverage
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/test/resources/org/apache/kylin/metadata/kylin_env.properties
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/resources/org/apache/kylin/metadata/kylin_env.properties b/core-metadata/src/test/resources/org/apache/kylin/metadata/kylin_env.properties
new file mode 100644
index 0000000..b9acf90
--- /dev/null
+++ b/core-metadata/src/test/resources/org/apache/kylin/metadata/kylin_env.properties
@@ -0,0 +1,5 @@
+PROD=/etc/kylin
+QA=/etc/kylin
+DEV=/etc/kylin
+
+YADEV=hbase:yadesk00:2181:/hbase-unsecure
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/core-metadata/src/test/test.json
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/test.json b/core-metadata/src/test/test.json
new file mode 100644
index 0000000..9811a33
--- /dev/null
+++ b/core-metadata/src/test/test.json
@@ -0,0 +1,181 @@
+{"id": 2, "name": "STAR_ITEM_CUBE", "status": "active", "fact_table": {
+    "name": "ITEM",
+    "type": "fact",
+    "database": "EDW"
+}, "dimensions": [
+    {
+        "id": 1,
+        "name": "CALENDAR",
+        "column": null,
+        "join": {
+            "type": "inner",
+            "primary_key": [
+                {
+                    "id": null,
+                    "name": "ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ],
+            "foreign_key": [
+                {
+                    "id": null,
+                    "name": "CALENDAR_ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ]
+        },
+        "hierarchy": [
+            {
+                "level": "1",
+                "column": {
+                    "id": null,
+                    "name": "CAL_YEAR",
+                    "datatype": "string",
+                    "table": {
+                        "name": "CALENDAR",
+                        "type": "lookup",
+                        "database": null
+                    }
+                }
+            }
+        ],
+        "table": {
+            "name": "CALENDAR",
+            "type": "lookup",
+            "database": null
+        },
+        "datatype": "string"
+    },
+    {
+        "id": 2,
+        "name": "SITE",
+        "column": {
+            "id": null,
+            "name": "SITE",
+            "datatype": "string",
+            "table": {
+                "name": "SITE",
+                "type": "lookup",
+                "database": null
+            }
+        },
+        "join": {
+            "type": "inner",
+            "primary_key": [
+                {
+                    "id": null,
+                    "name": "ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ],
+            "foreign_key": [
+                {
+                    "id": null,
+                    "name": "SITE_ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ]
+        },
+        "hierarchy": null,
+        "table": {
+            "name": "SITE",
+            "type": "lookup",
+            "database": null
+        },
+        "datatype": "string"
+    },
+    {
+        "id": 3,
+        "name": "CATEGORY",
+        "column": {
+            "id": null,
+            "name": "CATEGORY",
+            "datatype": "string",
+            "table": {
+                "name": "CATEGORY",
+                "type": "lookup",
+                "database": null
+            }
+        },
+        "join": {
+            "type": "inner",
+            "primary_key": [
+                {
+                    "id": null,
+                    "name": "ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ],
+            "foreign_key": [
+                {
+                    "id": null,
+                    "name": "CATEGORY_ID",
+                    "datatype": null,
+                    "table": null
+                }
+            ]
+        },
+        "hierarchy": null,
+        "table": {
+            "name": "CATEGORY",
+            "type": "lookup",
+            "database": null
+        },
+        "datatype": "string"
+    }
+], "measures": [
+    {
+        "id": 1,
+        "name": "USDAMTSUM",
+        "function": {
+            "expression": "SUM",
+            "parameter": {
+                "type": "column",
+                "value": "USD_AMT"
+            },
+            "returntype": "double"
+        }
+    },
+    {
+        "id": 2,
+        "name": "ITEMCOUNT",
+        "function": {
+            "expression": "SUM",
+            "parameter": {
+                "type": "column",
+                "value": "LISTING_CNT"
+            },
+            "returntype": "long"
+        }
+    }
+], "hbase_mapping": {
+    "type": "full_materialization",
+    "hbase_table": {
+        "name": "SAMPLE_CUBE",
+        "type": "hbase_table",
+        "database": "DEFAULT"
+    },
+    "rowkey_order": ["CAL_YEAR", "SITE", "CATEGORY"],
+    "rowkey_length": ["4", "2", "4"],
+    "high_cardinality_columns": null,
+    "column_family": [
+        {
+            "name": "CF1",
+            "columns": [
+                {
+                    "qualifier": "USD_AMT",
+                    "measure_refs": ["USDAMTSUM"]
+                },
+                {
+                    "qualifier": "ITEM_COUNT",
+                    "measure_refs": ["ITEMCOUNT"]
+                }
+            ]
+        }
+    ]
+}}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/cube/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/cube/.settings/org.eclipse.core.resources.prefs b/cube/.settings/org.eclipse.core.resources.prefs
deleted file mode 100644
index 04cfa2c..0000000
--- a/cube/.settings/org.eclipse.core.resources.prefs
+++ /dev/null
@@ -1,6 +0,0 @@
-eclipse.preferences.version=1
-encoding//src/main/java=UTF-8
-encoding//src/main/resources=UTF-8
-encoding//src/test/java=UTF-8
-encoding//src/test/resources=UTF-8
-encoding/<project>=UTF-8

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/cube/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/cube/.settings/org.eclipse.jdt.core.prefs b/cube/.settings/org.eclipse.jdt.core.prefs
deleted file mode 100644
index a903301..0000000
--- a/cube/.settings/org.eclipse.jdt.core.prefs
+++ /dev/null
@@ -1,379 +0,0 @@
-eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
-org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
-org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
-org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
-org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
-org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
-org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
-org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
-org.eclipse.jdt.core.compiler.compliance=1.7
-org.eclipse.jdt.core.compiler.debug.lineNumber=generate
-org.eclipse.jdt.core.compiler.debug.localVariable=generate
-org.eclipse.jdt.core.compiler.debug.sourceFile=generate
-org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
-org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
-org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
-org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
-org.eclipse.jdt.core.compiler.problem.deadCode=warning
-org.eclipse.jdt.core.compiler.problem.deprecation=warning
-org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
-org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
-org.eclipse.jdt.core.compiler.problem.discouragedReference=warning
-org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
-org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
-org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
-org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
-org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
-org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
-org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
-org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
-org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
-org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
-org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
-org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
-org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
-org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
-org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
-org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
-org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
-org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
-org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
-org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
-org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
-org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
-org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
-org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
-org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
-org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
-org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
-org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
-org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
-org.eclipse.jdt.core.compiler.problem.nullReference=warning
-org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
-org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
-org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
-org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
-org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
-org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
-org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
-org.eclipse.jdt.core.compiler.problem.rawTypeReference=warning
-org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
-org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
-org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
-org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
-org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
-org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
-org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
-org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
-org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
-org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
-org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
-org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
-org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
-org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
-org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=warning
-org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
-org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
-org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
-org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
-org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
-org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
-org.eclipse.jdt.core.compiler.problem.unusedImport=warning
-org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
-org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
-org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
-org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
-org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
-org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
-org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
-org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
-org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
-org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
-org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
-org.eclipse.jdt.core.compiler.source=1.7
-org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
-org.eclipse.jdt.core.formatter.alignment_for_assignment=0
-org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
-org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
-org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
-org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
-org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
-org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
-org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
-org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
-org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
-org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
-org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
-org.eclipse.jdt.core.formatter.blank_lines_after_package=1
-org.eclipse.jdt.core.formatter.blank_lines_before_field=0
-org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
-org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
-org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
-org.eclipse.jdt.core.formatter.blank_lines_before_method=1
-org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
-org.eclipse.jdt.core.formatter.blank_lines_before_package=0
-org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
-org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
-org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
-org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
-org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
-org.eclipse.jdt.core.formatter.comment.format_block_comments=false
-org.eclipse.jdt.core.formatter.comment.format_header=false
-org.eclipse.jdt.core.formatter.comment.format_html=true
-org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
-org.eclipse.jdt.core.formatter.comment.format_line_comments=false
-org.eclipse.jdt.core.formatter.comment.format_source_code=true
-org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
-org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
-org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
-org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
-org.eclipse.jdt.core.formatter.comment.line_length=80
-org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
-org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
-org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
-org.eclipse.jdt.core.formatter.compact_else_if=true
-org.eclipse.jdt.core.formatter.continuation_indentation=2
-org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
-org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
-org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
-org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
-org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
-org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
-org.eclipse.jdt.core.formatter.indent_empty_lines=false
-org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
-org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
-org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
-org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
-org.eclipse.jdt.core.formatter.indentation.size=4
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
-org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
-org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
-org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
-org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
-org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
-org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.join_lines_in_comments=true
-org.eclipse.jdt.core.formatter.join_wrapped_lines=true
-org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
-org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
-org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
-org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
-org.eclipse.jdt.core.formatter.lineSplit=999
-org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
-org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
-org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
-org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
-org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
-org.eclipse.jdt.core.formatter.tabulation.char=space
-org.eclipse.jdt.core.formatter.tabulation.size=4
-org.eclipse.jdt.core.formatter.use_on_off_tags=false
-org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
-org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
-org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
-org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/cube/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/cube/.settings/org.eclipse.jdt.ui.prefs b/cube/.settings/org.eclipse.jdt.ui.prefs
deleted file mode 100644
index dece0e6..0000000
--- a/cube/.settings/org.eclipse.jdt.ui.prefs
+++ /dev/null
@@ -1,7 +0,0 @@
-eclipse.preferences.version=1
-formatter_profile=_Space Indent & Long Lines
-formatter_settings_version=12
-org.eclipse.jdt.ui.ignorelowercasenames=true
-org.eclipse.jdt.ui.importorder=java;javax;org;com;
-org.eclipse.jdt.ui.ondemandthreshold=99
-org.eclipse.jdt.ui.staticondemandthreshold=1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/cube/pom.xml
----------------------------------------------------------------------
diff --git a/cube/pom.xml b/cube/pom.xml
deleted file mode 100644
index 07f9a58..0000000
--- a/cube/pom.xml
+++ /dev/null
@@ -1,186 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one
- or more contributor license agreements.  See the NOTICE file
- distributed with this work for additional information
- regarding copyright ownership.  The ASF licenses this file
- to you under the Apache License, Version 2.0 (the
- "License"); you may not use this file except in compliance
- with the License.  You may obtain a copy of the License at
- 
-     http://www.apache.org/licenses/LICENSE-2.0
- 
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-
-<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-cube</artifactId>
-    <packaging>jar</packaging>
-    <name>Kylin:Cube</name>
-
-    <parent>
-        <groupId>org.apache.kylin</groupId>
-        <artifactId>kylin</artifactId>
-        <version>0.7.2-incubating-SNAPSHOT</version>
-    </parent>
-
-    <dependencies>
-
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-common</artifactId>
-            <type>test-jar</type>
-            <scope>test</scope>
-            <version>${project.parent.version}</version>
-        </dependency>
-
-        <!--Kylin Jar -->
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-metadata</artifactId>
-            <version>${project.parent.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-dictionary</artifactId>
-            <version>${project.parent.version}</version>
-        </dependency>
-
-        <dependency>
-            <groupId>commons-cli</groupId>
-            <artifactId>commons-cli</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-lang</groupId>
-            <artifactId>commons-lang</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-configuration</groupId>
-            <artifactId>commons-configuration</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>log4j</groupId>
-            <artifactId>log4j</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.fasterxml.jackson.core</groupId>
-            <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>
-        <dependency>
-            <groupId>com.ning</groupId>
-            <artifactId>compress-lzf</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.n3twork.druid</groupId>
-            <artifactId>extendedset</artifactId>
-        </dependency>
-
-        <!-- Env & Test -->
-
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-common</artifactId>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-annotations</artifactId>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-mapreduce-client-core</artifactId>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-minicluster</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.mrunit</groupId>
-            <artifactId>mrunit</artifactId>
-            <classifier>hadoop2</classifier>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-hadoop2-compat</artifactId>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-common</artifactId>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-client</artifactId>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-server</artifactId>
-            <scope>provided</scope>
-            <!-- version conflict with hadoop2.2 -->
-            <exclusions>
-                <exclusion>
-                    <groupId>org.apache.hadoop</groupId>
-                    <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-
-    <!--
-        <build>
-            <plugins>
-                <plugin>
-                    <artifactId>maven-assembly-plugin</artifactId>
-                    <executions>
-                        <execution>
-                            <id>jar-with-dependencies</id>
-                            <phase>package</phase>
-                            <goals>
-                                <goal>single</goal>
-                            </goals>
-                            <configuration>
-                                <descriptors>
-                                    <descriptor>src/main/assembly/assemble.xml</descriptor>
-                                </descriptors>
-                            </configuration>
-                        </execution>
-                    </executions>
-                </plugin>
-            </plugins>
-        </build>
-     -->
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/cube/src/main/assembly/assemble.xml
----------------------------------------------------------------------
diff --git a/cube/src/main/assembly/assemble.xml b/cube/src/main/assembly/assemble.xml
deleted file mode 100644
index c26c222..0000000
--- a/cube/src/main/assembly/assemble.xml
+++ /dev/null
@@ -1,19 +0,0 @@
-<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-          xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
-    <id>jar-with-dependencies</id>
-    <formats>
-        <format>jar</format>
-    </formats>
-    <includeBaseDirectory>false</includeBaseDirectory>
-    <dependencySets>
-        <dependencySet>
-            <outputDirectory>/</outputDirectory>
-            <useProjectArtifact>true</useProjectArtifact>
-            <unpack>true</unpack>
-            <scope>runtime</scope>
-            <excludes>
-                <exclude>javax.servlet*:*</exclude>
-            </excludes>
-        </dependencySet>
-    </dependencySets>
-</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e8896ac/cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java b/cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
deleted file mode 100644
index 21e2737..0000000
--- a/cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ /dev/null
@@ -1,147 +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.cube;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.DimensionDesc;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.JoinDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.realization.SQLDigest;
-
-/**
- */
-public class CubeCapabilityChecker {
-    private static final Logger logger = LoggerFactory.getLogger(CubeCapabilityChecker.class);
-
-    public static boolean check(CubeInstance cube, SQLDigest digest, boolean allowWeakMatch) {
-
-        // retrieve members from olapContext
-        Collection<TblColRef> dimensionColumns = CubeDimensionDeriver.getDimensionColumns(digest.groupbyColumns, digest.filterColumns);
-        Collection<FunctionDesc> functions = digest.aggregations;
-        Collection<TblColRef> metricsColumns = digest.metricColumns;
-        Collection<JoinDesc> joins = digest.joinDescs;
-
-        // match dimensions & aggregations & joins
-
-        boolean isOnline = cube.isReady();
-
-        boolean matchDimensions = isMatchedWithDimensions(dimensionColumns, cube);
-        boolean matchAggregation = isMatchedWithAggregations(functions, cube);
-        boolean matchJoin = isMatchedWithJoins(joins, cube);
-
-        // Some cubes are not "perfectly" match, but still save them in case of usage
-        if (allowWeakMatch && isOnline && matchDimensions && !matchAggregation && matchJoin) {
-            // sometimes metrics are indeed dimensions
-            // e.g. select min(cal_dt) from ..., where cal_dt is actually a dimension
-            if (isWeaklyMatchedWithAggregations(functions, metricsColumns, cube)) {
-                logger.info("Weakly matched cube found " + cube.getName());
-                return true;
-            }
-        }
-
-        if (!isOnline || !matchDimensions || !matchAggregation || !matchJoin) {
-            logger.info("Exclude cube " + cube.getName() + " because " + " isOnlne=" + isOnline + ",matchDimensions=" + matchDimensions + ",matchAggregation=" + matchAggregation + ",matchJoin=" + matchJoin);
-            return false;
-        }
-
-        return true;
-    }
-
-    private static boolean isMatchedWithDimensions(Collection<TblColRef> dimensionColumns, CubeInstance cube) {
-        CubeDesc cubeDesc = cube.getDescriptor();
-        boolean matchAgg = cubeDesc.listDimensionColumnsIncludingDerived().containsAll(dimensionColumns);
-        return matchAgg;
-    }
-
-    private static boolean isMatchedWithAggregations(Collection<FunctionDesc> aggregations, CubeInstance cube) {
-        CubeDesc cubeDesc = cube.getDescriptor();
-        boolean matchAgg = cubeDesc.listAllFunctions().containsAll(aggregations);
-        return matchAgg;
-    }
-
-    private static boolean isMatchedWithJoins(Collection<JoinDesc> joins, CubeInstance cube) {
-        CubeDesc cubeDesc = cube.getDescriptor();
-
-        List<JoinDesc> cubeJoins = new ArrayList<JoinDesc>(cubeDesc.getDimensions().size());
-        for (DimensionDesc d : cubeDesc.getDimensions()) {
-            if (d.getJoin() != null) {
-                cubeJoins.add(d.getJoin());
-            }
-        }
-        for (JoinDesc j : joins) {
-            // optiq engine can't decide which one is fk or pk
-            String pTable = j.getPrimaryKeyColumns()[0].getTable();
-            String factTable = cubeDesc.getFactTable();
-            if (factTable.equals(pTable)) {
-                j.swapPKFK();
-            }
-
-            // check primary key, all PK column should refer to same tale, the Fact Table of cube.
-            // Using first column's table name to check.
-            String fTable = j.getForeignKeyColumns()[0].getTable();
-            if (!factTable.equals(fTable)) {
-                logger.info("Fact Table" + factTable + " not matched in join: " + j + " on cube " + cube.getName());
-                return false;
-            }
-
-            // The hashcode() function of JoinDesc has been overwritten,
-            // which takes into consideration: pk,fk,jointype
-            if (!cubeJoins.contains(j)) {
-                logger.info("Query joins don't macth on cube " + cube.getName());
-                return false;
-            }
-        }
-        return true;
-    }
-
-    private static boolean isWeaklyMatchedWithAggregations(Collection<FunctionDesc> aggregations, Collection<TblColRef> metricColumns, CubeInstance cube) {
-        CubeDesc cubeDesc = cube.getDescriptor();
-        Collection<FunctionDesc> cubeFuncs = cubeDesc.listAllFunctions();
-
-        boolean matched = true;
-        for (FunctionDesc functionDesc : aggregations) {
-            if (cubeFuncs.contains(functionDesc))
-                continue;
-
-            // only inverted-index cube does not have count, and let calcite handle in this case
-            if (functionDesc.isCount())
-                continue;
-
-            if (functionDesc.isCountDistinct()) // calcite can not handle distinct count
-                matched = false;
-
-            TblColRef col = null;
-            if (functionDesc.getParameter().getColRefs().size() > 0)
-                col = functionDesc.getParameter().getColRefs().get(0);
-
-            if (col == null || !cubeDesc.listDimensionColumnsIncludingDerived().contains(col)) {
-                matched = false;
-            }
-        }
-        return matched;
-    }
-}