You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by lu...@apache.org on 2015/01/07 15:46:58 UTC

[37/51] [partial] incubator-kylin git commit: migrate repo from github.com to apache git

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/kv/RowKeyEncoderTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/kv/RowKeyEncoderTest.java b/cube/src/test/java/com/kylinolap/cube/kv/RowKeyEncoderTest.java
new file mode 100644
index 0000000..a89e04f
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/kv/RowKeyEncoderTest.java
@@ -0,0 +1,143 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.cube.kv;
+
+import static org.junit.Assert.*;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.cuboid.Cuboid;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class RowKeyEncoderTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testEncodeWithoutSlr() throws Exception {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("TEST_KYLIN_CUBE_WITHOUT_SLR_READY");
+        // CubeSegment seg = cube.getTheOnlySegment();
+        CubeDesc cubeDesc = cube.getDescriptor();
+        // String data =
+        // "2013-08-18Abbigliamento e accessoriDonna: AccessoriSciarpFoulard e ScialliAuctionItalyRegular";
+        byte[][] data = new byte[8][];
+        data[0] = Bytes.toBytes("2012-12-15");
+        data[1] = Bytes.toBytes("11848");
+        data[2] = Bytes.toBytes("Health & Beauty");
+        data[3] = Bytes.toBytes("Fragrances");
+        data[4] = Bytes.toBytes("Women");
+        data[5] = Bytes.toBytes("FP-GTC");
+        data[6] = Bytes.toBytes("0");
+        data[7] = Bytes.toBytes("15");
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+        AbstractRowKeyEncoder rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cube.getFirstSegment(), baseCuboid);
+
+        byte[] encodedKey = rowKeyEncoder.encode(data);
+        assertEquals(30, encodedKey.length);
+        byte[] cuboidId = Arrays.copyOfRange(encodedKey, 0, 8);
+        byte[] rest = Arrays.copyOfRange(encodedKey, 8, encodedKey.length);
+        assertEquals(255, Bytes.toLong(cuboidId));
+        assertArrayEquals(new byte[] { 11, 55, -13, 13, 22, 34, 121, 70, 80, 45, 71, 84, 67, 9, 9, 9, 9, 9, 9, 0, 10, 5 }, rest);
+    }
+
+    @Test
+    public void testEncodeWithSlr() throws Exception {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("TEST_KYLIN_CUBE_WITH_SLR_READY");
+        // CubeSegment seg = cube.getTheOnlySegment();
+        CubeDesc cubeDesc = cube.getDescriptor();
+        // String data =
+        // "1234567892013-08-18Abbigliamento e accessoriDonna: AccessoriSciarpFoulard e ScialliAuctionItalyRegular";
+        byte[][] data = new byte[9][];
+        data[0] = Bytes.toBytes("123456789");
+        data[1] = Bytes.toBytes("2012-12-15");
+        data[2] = Bytes.toBytes("11848");
+        data[3] = Bytes.toBytes("Health & Beauty");
+        data[4] = Bytes.toBytes("Fragrances");
+        data[5] = Bytes.toBytes("Women");
+        data[6] = Bytes.toBytes("FP-GTC");
+        data[7] = Bytes.toBytes("0");
+        data[8] = Bytes.toBytes("15");
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+        AbstractRowKeyEncoder rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cube.getFirstSegment(), baseCuboid);
+
+        byte[] encodedKey = rowKeyEncoder.encode(data);
+        assertEquals(48, encodedKey.length);
+        byte[] sellerId = Arrays.copyOfRange(encodedKey, 8, 26);
+        byte[] cuboidId = Arrays.copyOfRange(encodedKey, 0, 8);
+        byte[] rest = Arrays.copyOfRange(encodedKey, 26, encodedKey.length);
+        assertTrue(Bytes.toString(sellerId).startsWith("123456789"));
+        assertEquals(511, Bytes.toLong(cuboidId));
+        assertArrayEquals(new byte[] { 11, 55, -13, 13, 22, 34, 121, 70, 80, 45, 71, 84, 67, 9, 9, 9, 9, 9, 9, 0, 10, 5 }, rest);
+    }
+
+    @Test
+    public void testEncodeWithSlr2() throws Exception {
+        CubeInstance cube = CubeManager.getInstance(this.getTestConfig()).getCube("TEST_KYLIN_CUBE_WITH_SLR_READY");
+        // CubeSegment seg = cube.getTheOnlySegment();
+        CubeDesc cubeDesc = cube.getDescriptor();
+        // String data =
+        // "1234567892013-08-18Abbigliamento e accessoriDonna: AccessoriSciarpFoulard e ScialliAuctionItalyRegular";
+        byte[][] data = new byte[9][];
+        data[0] = Bytes.toBytes("123456789");
+        data[1] = null;
+        data[2] = null;
+        data[3] = null;
+        data[4] = null;
+        data[5] = null;
+        data[6] = null;
+        data[7] = null;
+        data[8] = null;
+
+        long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+        Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+        AbstractRowKeyEncoder rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cube.getFirstSegment(), baseCuboid);
+
+        byte[] encodedKey = rowKeyEncoder.encode(data);
+        assertEquals(48, encodedKey.length);
+        byte[] sellerId = Arrays.copyOfRange(encodedKey, 8, 26);
+        byte[] cuboidId = Arrays.copyOfRange(encodedKey, 0, 8);
+        byte[] rest = Arrays.copyOfRange(encodedKey, 26, encodedKey.length);
+        assertTrue(Bytes.toString(sellerId).startsWith("123456789"));
+        assertEquals(511, Bytes.toLong(cuboidId));
+        assertArrayEquals(new byte[] { -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1 }, rest);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/kv/RowValueDecoderTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/kv/RowValueDecoderTest.java b/cube/src/test/java/com/kylinolap/cube/kv/RowValueDecoderTest.java
new file mode 100644
index 0000000..6595adc
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/kv/RowValueDecoderTest.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.cube.kv;
+
+import static org.junit.Assert.*;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.io.LongWritable;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.measure.MeasureCodec;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.HBaseColumnDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+
+/**
+ * @author George Song (ysong1)
+ * 
+ */
+public class RowValueDecoderTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testDecode() throws Exception {
+        CubeDesc cubeDesc = CubeManager.getInstance(this.getTestConfig()).getCube("test_kylin_cube_with_slr_ready").getDescriptor();
+        HBaseColumnDesc hbaseCol = cubeDesc.getHBaseMapping().getColumnFamily()[0].getColumns()[0];
+
+        MeasureCodec codec = new MeasureCodec(hbaseCol.getMeasures());
+        BigDecimal sum = new BigDecimal("333.1234567");
+        BigDecimal min = new BigDecimal("333.1111111");
+        BigDecimal max = new BigDecimal("333.1999999");
+        LongWritable count = new LongWritable(2);
+        ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+        codec.encode(new Object[] { sum, min, max, count }, buf);
+
+        buf.flip();
+        byte[] valueBytes = new byte[buf.limit()];
+        System.arraycopy(buf.array(), 0, valueBytes, 0, buf.limit());
+
+        RowValueDecoder rowValueDecoder = new RowValueDecoder(hbaseCol);
+        for (MeasureDesc measure : cubeDesc.getMeasures()) {
+            FunctionDesc aggrFunc = measure.getFunction();
+            int index = hbaseCol.findMeasureIndex(aggrFunc);
+            rowValueDecoder.setIndex(index);
+        }
+
+        rowValueDecoder.decode(valueBytes);
+        List<String> measureNames = rowValueDecoder.getNames();
+        Object[] measureValues = rowValueDecoder.getValues();
+
+        assertEquals("[PRICE, MIN_PRICE_, MAX_PRICE_, COUNT__]", measureNames.toString());
+        assertEquals("[333.1234567, 333.1111111, 333.1999999, 2]", Arrays.toString(measureValues));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/measure/MeasureCodecTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/measure/MeasureCodecTest.java b/cube/src/test/java/com/kylinolap/cube/measure/MeasureCodecTest.java
new file mode 100644
index 0000000..e3e78c9
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/measure/MeasureCodecTest.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.cube.measure;
+
+import static org.junit.Assert.*;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.junit.Test;
+
+import com.kylinolap.common.hll.HyperLogLogPlusCounter;
+import com.kylinolap.cube.kv.RowConstants;
+import com.kylinolap.metadata.model.cube.FunctionDesc;
+import com.kylinolap.metadata.model.cube.MeasureDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class MeasureCodecTest {
+
+    @Test
+    public void basicTest() {
+        MeasureDesc descs[] = new MeasureDesc[] { measure("double"), measure("long"), measure("decimal"), measure("HLLC16"), measure("HLLC16") };
+        MeasureCodec codec = new MeasureCodec(descs);
+
+        DoubleWritable d = new DoubleWritable(1.0);
+        LongWritable l = new LongWritable(2);
+        BigDecimal b = new BigDecimal("333.1234567");
+        HyperLogLogPlusCounter hllc = new HyperLogLogPlusCounter(16);
+        hllc.add("1234567");
+        hllc.add("abcdefg");
+        HyperLogLogPlusCounter hllc2 = new HyperLogLogPlusCounter(16);
+        hllc.add("1234567");
+        hllc.add("abcdefg");
+        Object values[] = new Object[] { d, l, b, hllc, hllc2 };
+
+        ByteBuffer buf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE);
+
+        codec.encode(values, buf);
+        buf.flip();
+        System.out.println("size: " + buf.limit());
+
+        Object copy[] = new Object[values.length];
+        codec.decode(buf, copy);
+
+        assertTrue(Arrays.equals(values, copy));
+    }
+
+    private MeasureDesc measure(String returnType) {
+        MeasureDesc desc = new MeasureDesc();
+        FunctionDesc func = new FunctionDesc();
+        func.setReturnType(returnType);
+        desc.setFunction(func);
+        return desc;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/java/com/kylinolap/cube/project/ProjectManagerTest.java
----------------------------------------------------------------------
diff --git a/cube/src/test/java/com/kylinolap/cube/project/ProjectManagerTest.java b/cube/src/test/java/com/kylinolap/cube/project/ProjectManagerTest.java
new file mode 100644
index 0000000..532c2dc
--- /dev/null
+++ b/cube/src/test/java/com/kylinolap/cube/project/ProjectManagerTest.java
@@ -0,0 +1,165 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.cube.project;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.common.util.LocalFileMetadataTestCase;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+
+/**
+ * @author xduo
+ * 
+ */
+public class ProjectManagerTest extends LocalFileMetadataTestCase {
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        MetadataManager.removeInstance(this.getTestConfig());
+        CubeManager.removeInstance(this.getTestConfig());
+        ProjectManager.removeInstance(this.getTestConfig());
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testDropNonemptyProject1() throws IOException {
+        ProjectManager.getInstance(this.getTestConfig()).dropProject("DEFAULT");
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testDropNonemptyProject2() throws IOException {
+        ProjectManager.getInstance(this.getTestConfig()).dropProject("DEFAULT???");
+    }
+
+    @Test
+    public void testNewProject() throws Exception {
+        int originalProjectCount = ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size();
+        int originalCubeCount = CubeManager.getInstance(this.getTestConfig()).listAllCubes().size();
+        int originalCubeCountInDefault = ProjectManager.getInstance(this.getTestConfig()).listAllCubes("default").size();
+        ResourceStore store = getStore();
+
+        // clean legacy in case last run failed
+        store.deleteResource("/cube/cube_in_alien_project.json");
+
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_with_slr_desc");
+        CubeInstance createdCube = CubeManager.getInstance(this.getTestConfig()).createCube("cube_in_alien_project", "alien", desc, null);
+        assertTrue(createdCube == CubeManager.getInstance(this.getTestConfig()).getCube("cube_in_alien_project"));
+        assertTrue(ProjectManager.getInstance(getTestConfig()).listAllCubes("alien").contains(createdCube));
+
+        System.out.println(JsonUtil.writeValueAsIndentString(createdCube));
+
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size() == originalProjectCount + 1);
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("ALIEN").get(0).getName().equalsIgnoreCase("CUBE_IN_ALIEN_PROJECT"));
+        assertTrue(CubeManager.getInstance(this.getTestConfig()).listAllCubes().size() == originalCubeCount + 1);
+
+        ProjectManager.getInstance(this.getTestConfig()).updateCubeToProject("cube_in_alien_project", "default", null);
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("ALIEN").size() == 0);
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("default").size() == originalCubeCountInDefault + 1);
+        assertTrue(ProjectManager.getInstance(getTestConfig()).listAllCubes("default").contains(createdCube));
+
+        ProjectManager.getInstance(this.getTestConfig()).updateCubeToProject("cube_in_alien_project", "alien", null);
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("ALIEN").size() == 1);
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("default").size() == originalCubeCountInDefault);
+        assertTrue(ProjectManager.getInstance(getTestConfig()).listAllCubes("alien").contains(createdCube));
+
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).isCubeInProject("alien", createdCube));
+
+        CubeInstance droppedCube = CubeManager.getInstance(this.getTestConfig()).dropCube("cube_in_alien_project", true);
+
+        assertTrue(createdCube == droppedCube);
+        assertNull(CubeManager.getInstance(this.getTestConfig()).getCube("cube_in_alien_project"));
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size() == originalProjectCount + 1);
+        assertTrue(CubeManager.getInstance(this.getTestConfig()).listAllCubes().size() == originalCubeCount);
+
+        ProjectManager.getInstance(this.getTestConfig()).dropProject("alien");
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size() == originalProjectCount);
+    }
+
+    @Test
+    public void testExistingProject() throws Exception {
+        int originalProjectCount = ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size();
+        int originalCubeCount = CubeManager.getInstance(this.getTestConfig()).listAllCubes().size();
+        ResourceStore store = getStore();
+
+        // clean legacy in case last run failed
+        store.deleteResource("/cube/new_cube_in_default.json");
+
+        MetadataManager metaMgr = getMetadataManager();
+        CubeDesc desc = metaMgr.getCubeDesc("test_kylin_cube_with_slr_desc");
+        CubeInstance createdCube = CubeManager.getInstance(this.getTestConfig()).createCube("new_cube_in_default", ProjectInstance.DEFAULT_PROJECT_NAME, desc, null);
+        assertTrue(createdCube == CubeManager.getInstance(this.getTestConfig()).getCube("new_cube_in_default"));
+
+        System.out.println(JsonUtil.writeValueAsIndentString(createdCube));
+
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size() == originalProjectCount);
+        assertTrue(CubeManager.getInstance(this.getTestConfig()).listAllCubes().size() == originalCubeCount + 1);
+
+        CubeInstance droppedCube = CubeManager.getInstance(this.getTestConfig()).dropCube("new_cube_in_default", true);
+
+        assertTrue(createdCube == droppedCube);
+        assertNull(CubeManager.getInstance(this.getTestConfig()).getCube("new_cube_in_default"));
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllProjects().size() == originalProjectCount);
+        assertTrue(CubeManager.getInstance(this.getTestConfig()).listAllCubes().size() == originalCubeCount);
+    }
+
+    @Test
+    public void testProjectsDrop() throws IOException {
+        CubeInstance cube = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_with_slr_empty");
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).getCubesByTable("default", "test_kylin_fact").contains(cube));
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).listAllCubes("default").contains(cube));
+
+        CubeManager.getInstance(getTestConfig()).dropCube(cube.getName(), true);
+
+        assertTrue(!ProjectManager.getInstance(this.getTestConfig()).getCubesByTable("default", "test_kylin_fact").contains(cube));
+        assertTrue(!ProjectManager.getInstance(this.getTestConfig()).listAllCubes("default").contains(cube));
+    }
+
+    @Test
+    public void testProjectsLoadAfterProjectChange() throws IOException {
+        CubeInstance cube = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_with_slr_empty");
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).getCubesByTable("default", "test_kylin_fact").contains(cube));
+
+        ProjectManager.getInstance(getTestConfig()).removeCubeFromProjects(cube.getName());
+
+        assertTrue(!ProjectManager.getInstance(this.getTestConfig()).getCubesByTable("default", "test_kylin_fact").contains(cube));
+
+        ProjectManager.getInstance(getTestConfig()).updateCubeToProject(cube.getName(), "default", "tester");
+
+        assertTrue(ProjectManager.getInstance(this.getTestConfig()).getCubesByTable("default", "test_kylin_fact").contains(cube));
+    }
+
+    private MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(getTestConfig());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/cube/src/test/resources/placeholder.txt
----------------------------------------------------------------------
diff --git a/cube/src/test/resources/placeholder.txt b/cube/src/test/resources/placeholder.txt
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/deploy.sh
----------------------------------------------------------------------
diff --git a/deploy.sh b/deploy.sh
new file mode 100755
index 0000000..4dd8355
--- /dev/null
+++ b/deploy.sh
@@ -0,0 +1,206 @@
+#!/usr/bin/env bash
+
+set -o pipefail  # trace ERR through pipes
+set -o errtrace  # trace ERR through 'time command' and other functions
+
+function error() {
+SCRIPT="$0"           # script name
+LASTLINE="$1"         # line of error occurrence
+LASTERR="$2"          # error code
+echo "ERROR exit from ${SCRIPT} : line ${LASTLINE} with exit code ${LASTERR}"
+exit 1
+}
+
+trap 'error ${LINENO} ${?}' ERR
+
+echo ""
+echo "Welcome to use Kylin-Deploy script"
+echo "This script will help you:"
+echo "1. Check environment"
+echo "2. Build Kylin artifacts"
+echo "3. Prepare test cube related data"
+echo "4. Lauch a web service to build cube and query with (at http://localhost:7070)"
+echo "Please make sure you are running this script on a hadoop CLI machine, and you have enough permissions."
+echo "Also, We assume you have installed: JAVA, TOMCAT, NPM and MAVEN."
+echo "[Warning] The installation may break existing tomcat applications on this CLI"
+echo ""
+
+
+[[ "$SILENT" ]] || ( read -p "Are you sure you want to proceed?(press Y or y to confirm) " -n 1 -r
+echo    # (optional) move to a new line
+if [[ ! $REPLY =~ ^[Yy]$ ]]
+then
+    echo "Not going to proceed, quit without finishing! You can rerun the script to have another try."
+    exit 1
+fi )
+
+echo "Checking JAVA status..."
+
+if [ -z "$JAVA_HOME" ]
+then
+    echo "Please set JAVA_HOME so that Kylin-Deploy can proceed"
+    exit 1
+else
+    echo "JAVA_HOME is set to $JAVA_HOME"
+fi
+
+if [ -d "$JAVA_HOME" ]
+then
+    echo "$JAVA_HOME exists"
+else
+    echo " $JAVA_HOME does not exist or is not a directory."
+    exit 1
+fi
+
+echo "Checking tomcat status..."
+
+if [ -z "$CATALINA_HOME" ]
+then
+    echo "Please set CATALINA_HOME so that Kylin-Deploy knows where to start tomcat"
+    exit 1
+else
+    echo "CATALINA_HOME is set to $CATALINA_HOME"
+fi
+
+if [ -d "$CATALINA_HOME" ]
+then
+    echo "$CATALINA_HOME exists"
+else
+    echo " $CATALINA_HOME does not exist or is not a directory."
+    exit 1
+fi
+
+echo "Checking maven..."
+
+if [ -z "$(command -v mvn)" ]
+then
+    echo "Please install maven first so that Kylin-Deploy can proceed"
+    exit 1
+else
+    echo "maven check passed"
+fi
+
+echo "Checking npm..."
+
+if [ -z "$(command -v npm)" ]
+then
+    echo "Please install npm first so that Kylin-Deploy can proceed"
+    exit 1
+else
+    echo "npm check passed"
+fi
+
+KYLIN_HOME="$( cd "$( dirname "${BASH_SOURCE[0]}" )"  && pwd )"
+echo "Kylin home folder path is $KYLIN_HOME"
+cd $KYLIN_HOME
+
+
+echo "Building and packaging..."
+source ./package.sh
+
+echo "retrieving classpath..."
+cd $KYLIN_HOME/job/target
+JOB_JAR_NAME="kylin-job-latest.jar"
+#generate config variables
+hbase org.apache.hadoop.util.RunJar $JOB_JAR_NAME com.kylinolap.job.deployment.HbaseConfigPrinter /tmp/kylin_retrieve.sh
+#load config variables
+source /tmp/kylin_retrieve.sh
+
+cd $KYLIN_HOME
+mkdir -p /etc/kylin
+
+HOSTNAME=`hostname`
+CLI_HOSTNAME_DEFAULT="kylin.job.remote.cli.hostname=sandbox.hortonworks.com"
+CLI_PASSWORD_DEFAULT="kylin.job.remote.cli.password=hadoop"
+METADATA_URL_DEFAULT="kylin.metadata.url=kylin_metadata_qa@hbase:sandbox.hortonworks.com:2181:/hbase-unsecure"
+STORAGE_URL_DEFAULT="kylin.storage.url=hbase:sandbox.hortonworks.com:2181:/hbase-unsecure"
+CHECK_URL_DEFAULT="kylin.job.yarn.app.rest.check.status.url=http://sandbox"
+
+
+NEW_CLI_HOSTNAME_PREFIX="kylin.job.remote.cli.hostname="
+NEW_CLI_PASSWORD_PREFIX="kylin.job.remote.cli.password="
+NEW_METADATA_URL_PREFIX="kylin.metadata.url=kylin_metadata_qa@hbase:"
+NEW_STORAGE_URL_PREFIX="kylin.storage.url=hbase:"
+NEW_CHECK_URL_PREFIX="kylin.job.yarn.app.rest.check.status.url=http://"
+
+KYLIN_ZOOKEEPER_URL=${KYLIN_ZOOKEEPER_QUORUM}:${KYLIN_ZOOKEEPER_CLIENT_PORT}:${KYLIN_ZOOKEEPER_ZNODE_PARENT}
+
+echo "Kylin install script requires root password for ${HOSTNAME}"
+echo "(The default root password for hortonworks VM is hadoop, and for cloudera VM is cloudera)"
+
+[[ "$SILENT" ]] || read -s -p "Enter Password for root: " ROOTPASS
+
+#deploy kylin.properties to /etc/kylin
+cat examples/test_case_data/sandbox/kylin.properties | \
+    sed -e "s,${CHECK_URL_DEFAULT},${NEW_CHECK_URL_PREFIX}${HOSTNAME}," | \
+    sed -e "s,${CLI_HOSTNAME_DEFAULT},${NEW_CLI_HOSTNAME_PREFIX}${HOSTNAME}," | \
+    sed -e "s,${CLI_PASSWORD_DEFAULT},${NEW_CLI_PASSWORD_PREFIX}${ROOTPASS}," | \
+    sed -e "s,${METADATA_URL_DEFAULT},${NEW_METADATA_URL_PREFIX}${KYLIN_ZOOKEEPER_URL}," | \
+    sed -e "s,${STORAGE_URL_DEFAULT},${NEW_STORAGE_URL_PREFIX}${KYLIN_ZOOKEEPER_URL}," >  /etc/kylin/kylin.properties
+
+
+echo "a copy of kylin config is generated at /etc/kylin/kylin.properties:"
+echo "==================================================================="
+cat /etc/kylin/kylin.properties
+echo ""
+echo "==================================================================="
+echo ""
+
+[[ "$SILENT" ]] || ( read -p "please ensure the CLI address/username/password is correct, and press y to proceed: " -n 1 -r
+echo    # (optional) move to a new line
+if [[ ! $REPLY =~ ^[Yy]$ ]]
+then
+    echo "Not going to proceed, quit without finishing! You can rerun the script to have another try."
+    exit 1
+fi )
+
+# 1. generate synthetic fact table(test_kylin_fact) data and dump it into hive
+# 2. create empty cubes on these data, ready to be built
+cd $KYLIN_HOME
+mvn test -Dtest=com.kylinolap.job.SampleCubeSetupTest -DfailIfNoTests=false
+
+sudo -i "${CATALINA_HOME}/bin/shutdown.sh" || true # avoid trapping
+cd $KYLIN_HOME/server/target
+WAR_NAME="kylin.war"
+rm -rf $CATALINA_HOME/webapps/kylin
+rm -f $CATALINA_HOME/webapps/$WAR_NAME
+cp $KYLIN_HOME/server/target/$WAR_NAME $CATALINA_HOME/webapps/
+cd $CATALINA_HOME/webapps;
+chmod 644 $WAR_NAME;
+echo "REST service deployed"
+
+rm -rf /var/www/html/kylin
+mkdir -p /var/www/html/kylin
+cd $KYLIN_HOME/
+tar -xf webapp/dist/Web.tar -C /var/www/html/kylin
+echo "Web deployed"
+
+cd $KYLIN_HOME/
+#deploy setenv.sh
+rm -rf $CATALINA_HOME/bin/setenv.sh
+echo JAVA_OPTS=\"-Djava.library.path=${KYLIN_LD_LIBRARY_PATH}\" >> ${CATALINA_HOME}/bin/setenv.sh
+echo CATALINA_OPTS=\"-Dorg.apache.tomcat.util.buf.UDecoder.ALLOW_ENCODED_SLASH=true -Dorg.apache.catalina.connector.CoyoteAdapter.ALLOW_BACKSLASH=true -Dspring.profiles.active=sandbox \" >> ${CATALINA_HOME}/bin/setenv.sh
+echo CLASSPATH=\"${CATALINA_HOME}/lib/*:${KYLIN_HBASE_CLASSPATH}:/etc/kylin\" >> ${CATALINA_HOME}/bin/setenv.sh
+echo "setenv.sh created"
+
+#deploy server.xml
+rm -rf ${CATALINA_HOME}/conf/server.xml
+cp deploy/server.xml ${CATALINA_HOME}/conf/server.xml
+echo "server.xml copied"
+
+#deploy web.xml
+rm -rf ${CATALINA_HOME}/conf/web.xml
+cp deploy/web.xml ${CATALINA_HOME}/conf/web.xml
+echo "web.xml copied"
+
+echo "Tomcat ready"
+
+# redeploy coprocessor
+#hbase org.apache.hadoop.util.RunJar /usr/lib/kylin/kylin-job-latest.jar com.kylinolap.job.tools.DeployCoprocessorCLI /usr/lib/kylin/kylin-coprocessor-latest.jar
+
+
+sudo -i "${CATALINA_HOME}/bin/startup.sh"
+
+
+echo "Kylin-Deploy Success!"
+echo "Please visit http://<your_sandbox_ip>:7070 to play with the cubes! (Useranme: ADMIN, Password: KYLIN)"

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/deploy/install.sh
----------------------------------------------------------------------
diff --git a/deploy/install.sh b/deploy/install.sh
new file mode 100755
index 0000000..a8c7108
--- /dev/null
+++ b/deploy/install.sh
@@ -0,0 +1,15 @@
+#!/bin/sh
+
+
+cd ~
+wget http://apache.cs.uu.nl/dist/tomcat/tomcat-7/v7.0.56/bin/apache-tomcat-7.0.56.tar.gz
+tar -xzvf apache-tomcat-7.0.56.tar.gz
+export CATALINA_HOME=/root/apache-tomcat-7.0.56
+
+wget http://apache.proserve.nl/maven/maven-3/3.2.3/binaries/apache-maven-3.2.3-bin.tar.gz
+tar -xzvf apache-maven-3.2.3-bin.tar.gz
+ln -s /root/apache-maven-3.2.3/bin/mvn /usr/bin/mvn
+
+wget http://nodejs.org/dist/v0.10.32/node-v0.10.32-linux-x64.tar.gz
+tar -xzvf node-v0.10.32-linux-x64.tar.gz
+export PATH=/root/node-v0.10.32-linux-x64/bin:$PATH
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/deploy/server.xml
----------------------------------------------------------------------
diff --git a/deploy/server.xml b/deploy/server.xml
new file mode 100644
index 0000000..4bd26d0
--- /dev/null
+++ b/deploy/server.xml
@@ -0,0 +1,156 @@
+<?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.
+-->
+<!-- Note:  A "Server" is not itself a "Container", so you may not
+     define subcomponents such as "Valves" at this level.
+     Documentation at /docs/config/server.html
+ -->
+<Server port="9005" shutdown="SHUTDOWN">
+    <!-- Security listener. Documentation at /docs/config/listeners.html
+    <Listener className="org.apache.catalina.security.SecurityListener" />
+    -->
+    <!--APR library loader. Documentation at /docs/apr.html -->
+    <Listener className="org.apache.catalina.core.AprLifecycleListener" SSLEngine="on" />
+    <!--Initialize Jasper prior to webapps are loaded. Documentation at /docs/jasper-howto.html -->
+    <Listener className="org.apache.catalina.core.JasperListener" />
+    <!-- Prevent memory leaks due to use of particular java/javax APIs-->
+    <Listener className="org.apache.catalina.core.JreMemoryLeakPreventionListener" />
+    <Listener className="org.apache.catalina.mbeans.GlobalResourcesLifecycleListener" />
+    <Listener className="org.apache.catalina.core.ThreadLocalLeakPreventionListener" />
+
+    <!-- Global JNDI resources
+         Documentation at /docs/jndi-resources-howto.html
+    -->
+    <GlobalNamingResources>
+        <!-- Editable user database that can also be used by
+             UserDatabaseRealm to authenticate users
+        -->
+        <Resource name="UserDatabase" auth="Container"
+                  type="org.apache.catalina.UserDatabase"
+                  description="User database that can be updated and saved"
+                  factory="org.apache.catalina.users.MemoryUserDatabaseFactory"
+                  pathname="conf/tomcat-users.xml" />
+    </GlobalNamingResources>
+
+    <!-- A "Service" is a collection of one or more "Connectors" that share
+         a single "Container" Note:  A "Service" is not itself a "Container",
+         so you may not define subcomponents such as "Valves" at this level.
+         Documentation at /docs/config/service.html
+     -->
+    <Service name="Catalina">
+
+        <!--The connectors can use a shared executor, you can define one or more named thread pools-->
+        <!--
+        <Executor name="tomcatThreadPool" namePrefix="catalina-exec-"
+            maxThreads="150" minSpareThreads="4"/>
+        -->
+
+
+        <!-- A "Connector" represents an endpoint by which requests are received
+             and responses are returned. Documentation at :
+             Java HTTP Connector: /docs/config/http.html (blocking & non-blocking)
+             Java AJP  Connector: /docs/config/ajp.html
+             APR (HTTP/AJP) Connector: /docs/apr.html
+             Define a non-SSL HTTP/1.1 Connector on port 8080
+        -->
+        <Connector port="7070" protocol="HTTP/1.1"
+                   connectionTimeout="20000"
+                   redirectPort="9443"
+                   compression="on"
+                   compressionMinSize="2048"
+                   noCompressionUserAgents="gozilla,traviata"
+                   compressableMimeType="text/html,text/xml,text/javascript,application/javascript,application/json,text/css,text/plain"
+                />
+        <!-- A "Connector" using the shared thread pool-->
+        <!--
+        <Connector executor="tomcatThreadPool"
+                   port="8080" protocol="HTTP/1.1"
+                   connectionTimeout="20000"
+                   redirectPort="8443" />
+        -->
+        <!-- Define a SSL HTTP/1.1 Connector on port 8443
+             This connector uses the BIO implementation that requires the JSSE
+             style configuration. When using the APR/native implementation, the
+             OpenSSL style configuration is required as described in the APR/native
+             documentation -->
+        <!--
+        <Connector port="8443" protocol="org.apache.coyote.http11.Http11Protocol"
+                   maxThreads="150" SSLEnabled="true" scheme="https" secure="true"
+                   clientAuth="false" sslProtocol="TLS" />
+        -->
+
+        <!-- Define an AJP 1.3 Connector on port 8009 -->
+        <Connector port="9009" protocol="AJP/1.3" redirectPort="9443" />
+
+
+        <!-- An Engine represents the entry point (within Catalina) that processes
+             every request.  The Engine implementation for Tomcat stand alone
+             analyzes the HTTP headers included with the request, and passes them
+             on to the appropriate Host (virtual host).
+             Documentation at /docs/config/engine.html -->
+
+        <!-- You should set jvmRoute to support load-balancing via AJP ie :
+        <Engine name="Catalina" defaultHost="localhost" jvmRoute="jvm1">
+        -->
+        <Engine name="Catalina" defaultHost="localhost">
+
+            <!--For clustering, please take a look at documentation at:
+                /docs/cluster-howto.html  (simple how to)
+                /docs/config/cluster.html (reference documentation) -->
+            <!--
+            <Cluster className="org.apache.catalina.ha.tcp.SimpleTcpCluster"/>
+            -->
+
+            <!-- Use the LockOutRealm to prevent attempts to guess user passwords
+                 via a brute-force attack -->
+            <Realm className="org.apache.catalina.realm.LockOutRealm">
+                <!-- This Realm uses the UserDatabase configured in the global JNDI
+                     resources under the key "UserDatabase".  Any edits
+                     that are performed against this UserDatabase are immediately
+                     available for use by the Realm.  -->
+                <Realm className="org.apache.catalina.realm.UserDatabaseRealm"
+                       resourceName="UserDatabase"/>
+            </Realm>
+
+            <Host name="localhost"  appBase="webapps"
+                  unpackWARs="true" autoDeploy="true">
+
+                <!-- SingleSignOn valve, share authentication between web applications
+                     Documentation at: /docs/config/valve.html -->
+                <!--
+                <Valve className="org.apache.catalina.authenticator.SingleSignOn" />
+                -->
+
+                <!-- Access log processes all example.
+                     Documentation at: /docs/config/valve.html
+                     Note: The pattern used is equivalent to using pattern="common" -->
+                <Valve className="org.apache.catalina.valves.AccessLogValve" directory="logs"
+                       prefix="localhost_access_log." suffix=".txt"
+                       pattern="%h %l %u %t &quot;%r&quot; %s %b" />
+                <Context path="" docBase="/var/www/html/kylin">
+                    <!-- Default set of monitored resources -->
+                    <WatchedResource>WEB-INF/web.xml</WatchedResource>
+                </Context>
+                <Context path="/ROOT" docBase="ROOT">
+                    <!-- Default set of monitored resources -->
+                    <WatchedResource>WEB-INF/web.xml</WatchedResource>
+                </Context>
+
+            </Host>
+        </Engine>
+    </Service>
+</Server>