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

[48/50] [abbrv] incubator-kylin git commit: KYLIN-759 add test cases and bug fix

KYLIN-759 add test cases and bug fix


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

Branch: refs/heads/streaming-localdict
Commit: 0e42b2dfae6f490ae25e33241f3982f7de66fd59
Parents: bc426cc
Author: honma <ho...@ebay.com>
Authored: Thu May 14 13:57:31 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Thu May 14 20:22:04 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/util/IdentityUtils.java |  35 +++++
 .../org/apache/kylin/common/util/BasicTest.java |  24 ++--
 .../kylin/common/util/IdentityUtilTest.java     |  30 ++++
 .../apache/kylin/query/test/KylinQueryTest.java |   2 +-
 .../kylin/storage/ICachableStorageEngine.java   |   7 +
 .../cache/CacheFledgedDynamicStorageEngine.java |   5 +-
 .../cache/CacheFledgedStaticStorageEngine.java  |   2 +-
 .../kylin/storage/cache/StreamSQLResult.java    |   3 +
 .../org/apache/kylin/storage/tuple/Tuple.java   |   8 +-
 .../apache/kylin/storage/cache/EhcacheTest.java |   5 -
 .../kylin/storage/test/DynamicCacheTest.java    | 141 +++++++++++++++++++
 .../kylin/storage/test/StaticCacheTest.java     |  95 +++++++++++++
 .../kylin/storage/test/StorageMockUtils.java    | 132 +++++++++++++++++
 .../apache/kylin/storage/test/StorageTest.java  | 119 ++--------------
 14 files changed, 476 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/common/src/main/java/org/apache/kylin/common/util/IdentityUtils.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/util/IdentityUtils.java b/common/src/main/java/org/apache/kylin/common/util/IdentityUtils.java
new file mode 100644
index 0000000..d0691da
--- /dev/null
+++ b/common/src/main/java/org/apache/kylin/common/util/IdentityUtils.java
@@ -0,0 +1,35 @@
+package org.apache.kylin.common.util;
+
+import java.util.Collection;
+import java.util.IdentityHashMap;
+
+/**
+ * Created by Hongbin Ma(Binmahone) on 5/14/15.
+ */
+public class IdentityUtils {
+    public static <K> boolean collectionReferenceEquals(Collection<K> collectionA, Collection<K> collectionB) {
+        if (collectionA == null || collectionB == null) {
+            throw new RuntimeException("input must be not null");
+        }
+
+        IdentityHashMap<K, Void> mapA = new IdentityHashMap<>();
+        IdentityHashMap<K, Void> mapB = new IdentityHashMap<>();
+        for (K key : collectionA) {
+            mapA.put(key, null);
+        }
+        for (K key : collectionB) {
+            mapB.put(key, null);
+        }
+
+        if (mapA.keySet().size() != mapB.keySet().size()) {
+            return false;
+        }
+
+        for (K key : mapA.keySet()) {
+            if (!mapB.keySet().contains(key)) {
+                return false;
+            }
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/common/src/test/java/org/apache/kylin/common/util/BasicTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/kylin/common/util/BasicTest.java b/common/src/test/java/org/apache/kylin/common/util/BasicTest.java
index e0226f8..b8fb5e7 100644
--- a/common/src/test/java/org/apache/kylin/common/util/BasicTest.java
+++ b/common/src/test/java/org/apache/kylin/common/util/BasicTest.java
@@ -19,8 +19,8 @@
 package org.apache.kylin.common.util;
 
 import com.google.common.collect.Lists;
-import com.google.common.collect.Ranges;
 import org.apache.commons.configuration.ConfigurationException;
+import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.LoggerFactory;
@@ -31,6 +31,7 @@ import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Calendar;
+import java.util.IdentityHashMap;
 
 /**
 * <p/>
@@ -60,16 +61,21 @@ public class BasicTest {
     }
 
     @Test
+    public void test0() throws Exception {
+
+        IdentityHashMap<String, Void> a = new IdentityHashMap<>();
+        IdentityHashMap<String, Void> b = new IdentityHashMap<>();
+        String s1 = new String("s1");
+        String s2 = new String("s1");
+        Assert.assertEquals(s1, s2);
+        Assert.assertTrue(s1 != s2);
+        a.put(s1, null);
+        b.put(s2, null);
+    }
+
+    @Test
     @Ignore("convenient trial tool for dev")
     public void test1() throws Exception {
-        Number xx =new Long(0L);
-        System.out.println(xx.getClass().getName());
-
-        System.out.println(time(1367798400000L));
-
-        System.out.println(org.apache.kylin.common.util.DateFormat.formatToTimeStr(1000L * Integer.MAX_VALUE));
-
-        System.out.println(Ranges.open(3, 5).isConnected(Ranges.open(3, 10)));
 
         String bb = "\\x00\\x00\\x00\\x00\\x01\\x3F\\xD0\\x2D\\58\\x00\\x00\\x00\\x00\\x00\\x00\\x00\\x00\\x00\\x00\\x00";//2013/07/12 07:59:37
         String cc = "\\x00\\x00\\x00\\x00\\x01\\x41\\xBE\\x8F\\xD8\\x00\\x00\\x00\\x00\\x00\\x00\\x00";//2013/10/16 08:00:00

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/common/src/test/java/org/apache/kylin/common/util/IdentityUtilTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/kylin/common/util/IdentityUtilTest.java b/common/src/test/java/org/apache/kylin/common/util/IdentityUtilTest.java
new file mode 100644
index 0000000..99385f6
--- /dev/null
+++ b/common/src/test/java/org/apache/kylin/common/util/IdentityUtilTest.java
@@ -0,0 +1,30 @@
+package org.apache.kylin.common.util;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.collections.CollectionUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Created by Hongbin Ma(Binmahone) on 5/14/15.
+ */
+public class IdentityUtilTest {
+    @Test
+    public void basicTest()
+    {
+        String s1 = new String("hi");
+        String s2 = new String("hi");
+
+        List<String> c1 = Lists.newArrayList(s1);
+        List<String> c2 = Lists.newArrayList(s2);
+        List<String> c3 = Lists.newArrayList(s2);
+
+        Assert.assertTrue(CollectionUtils.isEqualCollection(c1,c2));
+        Assert.assertTrue(CollectionUtils.isEqualCollection(c3,c2));
+
+        Assert.assertFalse(IdentityUtils.collectionReferenceEquals(c1,c2));
+        Assert.assertTrue(IdentityUtils.collectionReferenceEquals(c3,c2));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java b/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
index 40372ff..cea8ee2 100644
--- a/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
+++ b/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
@@ -95,7 +95,7 @@ public class KylinQueryTest extends KylinTestBase {
     @Test
     public void testSingleRunQuery() throws Exception {
 
-        String queryFileName = "src/test/resources/query/sql_derived/query07.sql";
+        String queryFileName = "src/test/resources/query/sql/query00.sql";
 
         File sqlFile = new File(queryFileName);
         if (sqlFile.exists()) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/storage/src/main/java/org/apache/kylin/storage/ICachableStorageEngine.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/ICachableStorageEngine.java b/storage/src/main/java/org/apache/kylin/storage/ICachableStorageEngine.java
new file mode 100644
index 0000000..eadef09
--- /dev/null
+++ b/storage/src/main/java/org/apache/kylin/storage/ICachableStorageEngine.java
@@ -0,0 +1,7 @@
+package org.apache.kylin.storage;
+
+/**
+ * Created by Hongbin Ma(Binmahone) on 5/14/15.
+ */
+public interface ICachableStorageEngine extends IStorageEngine{
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicStorageEngine.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicStorageEngine.java b/storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicStorageEngine.java
index 0d7ab18..65672ae 100644
--- a/storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicStorageEngine.java
+++ b/storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicStorageEngine.java
@@ -41,7 +41,8 @@ public class CacheFledgedDynamicStorageEngine extends AbstractCacheFledgedStorag
 
     @Override
     public ITupleIterator search(final StorageContext context, final SQLDigest sqlDigest, final TupleInfo returnTupleInfo) {
-        //if isDynamicStorage && query involves filter on partition column, the cache requires updating after query done.
+        //enable dynamic cache iff group by columns contains partition col
+        //because cache extraction requires partition col value as selection key
         boolean needUpdateCache = sqlDigest.groupbyColumns.contains(partitionColRef);
 
         streamSQLDigest = new StreamSQLDigest(sqlDigest, partitionColRef);
@@ -106,7 +107,7 @@ public class CacheFledgedDynamicStorageEngine extends AbstractCacheFledgedStorag
             logger.info("decision: use cache");
         }
 
-        if (needUpdateCache || !queryCacheExists) {
+        if (needUpdateCache) {
             //use another nested ITupleIterator to deal with cache
             final TeeTupleIterator tee = new TeeTupleIterator(ret);
             tee.addCloseListener(this);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticStorageEngine.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticStorageEngine.java b/storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticStorageEngine.java
index a4aceed..860dea2 100644
--- a/storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticStorageEngine.java
+++ b/storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticStorageEngine.java
@@ -51,7 +51,7 @@ public class CacheFledgedStaticStorageEngine extends AbstractCacheFledgedStorage
             logger.info("decision: use cache");
         }
 
-        if (queryCacheExists) {
+        if (!queryCacheExists) {
             //use another nested ITupleIterator to deal with cache
             final TeeTupleIterator tee = new TeeTupleIterator(ret);
             tee.addCloseListener(this);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/storage/src/main/java/org/apache/kylin/storage/cache/StreamSQLResult.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/cache/StreamSQLResult.java b/storage/src/main/java/org/apache/kylin/storage/cache/StreamSQLResult.java
index e895d8e..924044b 100644
--- a/storage/src/main/java/org/apache/kylin/storage/cache/StreamSQLResult.java
+++ b/storage/src/main/java/org/apache/kylin/storage/cache/StreamSQLResult.java
@@ -39,6 +39,9 @@ public class StreamSQLResult {
                     }
                 }
             } else {
+                if (!this.sortedRows.containsKey(0L)) {
+                    this.sortedRows.put(0L, Lists.<ITuple> newArrayList());
+                }
                 this.sortedRows.get(0L).add(row);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java b/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
index 7224cfe..efa1bc2 100644
--- a/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
+++ b/storage/src/main/java/org/apache/kylin/storage/tuple/Tuple.java
@@ -135,7 +135,9 @@ public class Tuple implements ITuple {
         return 1L * days * (1000 * 3600 * 24);
     }
 
-    public static int millisToEpicDays(long millis) {
+    public static int dateToEpicDays(String strValue) {
+        Date dateValue = DateFormat.stringToDate(strValue); // NOTE: forces GMT timezone
+        long millis = dateValue.getTime();
         return (int) (millis / (1000 * 3600 * 24));
     }
 
@@ -158,9 +160,7 @@ public class Tuple implements ITuple {
         // TODO use data type enum instead of string comparison
         if ("date".equals(dataTypeName)) {
             // convert epoch time
-            Date dateValue = DateFormat.stringToDate(strValue); // NOTE: forces GMT timezone
-            long millis = dateValue.getTime();
-            return millisToEpicDays(millis);// Optiq expects Integer instead of Long. by honma
+            return dateToEpicDays(strValue);// Optiq expects Integer instead of Long. by honma
         } else if ("timestamp".equals(dataTypeName) || "datetime".equals(dataTypeName)) {
             return Long.parseLong(strValue);
         } else if ("tinyint".equals(dataTypeName)) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/storage/src/test/java/org/apache/kylin/storage/cache/EhcacheTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/cache/EhcacheTest.java b/storage/src/test/java/org/apache/kylin/storage/cache/EhcacheTest.java
index e86771d..1dec9e6 100644
--- a/storage/src/test/java/org/apache/kylin/storage/cache/EhcacheTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/cache/EhcacheTest.java
@@ -1,9 +1,7 @@
 package org.apache.kylin.storage.cache;
 
-import com.google.common.collect.Maps;
 import net.sf.ehcache.Cache;
 import net.sf.ehcache.CacheManager;
-import net.sf.ehcache.Element;
 import net.sf.ehcache.config.CacheConfiguration;
 import net.sf.ehcache.config.MemoryUnit;
 import net.sf.ehcache.config.PersistenceConfiguration;
@@ -11,9 +9,6 @@ import net.sf.ehcache.store.MemoryStoreEvictionPolicy;
 
 import org.junit.Test;
 
-import java.util.HashMap;
-import java.util.HashSet;
-
 /**
  */
 public class EhcacheTest {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/storage/src/test/java/org/apache/kylin/storage/test/DynamicCacheTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/test/DynamicCacheTest.java b/storage/src/test/java/org/apache/kylin/storage/test/DynamicCacheTest.java
new file mode 100644
index 0000000..0d962cb
--- /dev/null
+++ b/storage/src/test/java/org/apache/kylin/storage/test/DynamicCacheTest.java
@@ -0,0 +1,141 @@
+package org.apache.kylin.storage.test;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import com.google.common.collect.Ranges;
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.common.util.IdentityUtils;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.realization.SQLDigest;
+import org.apache.kylin.metadata.tuple.ITuple;
+import org.apache.kylin.metadata.tuple.ITupleIterator;
+import org.apache.kylin.metadata.tuple.SimpleTupleIterator;
+import org.apache.kylin.storage.IStorageEngine;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.cache.CacheFledgedDynamicStorageEngine;
+import org.apache.kylin.storage.hbase.coprocessor.endpoint.TsConditionExtractor;
+import org.apache.kylin.storage.tuple.Tuple;
+import org.apache.kylin.storage.tuple.TupleInfo;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Created by Hongbin Ma(Binmahone) on 5/14/15.
+ */
+public class DynamicCacheTest {
+
+    class TsOnlyTuple implements ITuple {
+        private TblColRef partitionCol;
+        private String tsStr;
+
+        public TsOnlyTuple(TblColRef partitionCol, String tsStr) {
+            this.partitionCol = partitionCol;
+            this.tsStr = tsStr;
+        }
+
+        @Override
+        public List<String> getAllFields() {
+            throw new NotImplementedException();
+        }
+
+        @Override
+        public List<TblColRef> getAllColumns() {
+            throw new NotImplementedException();
+        }
+
+        @Override
+        public Object[] getAllValues() {
+            throw new NotImplementedException();
+        }
+
+        @Override
+        public ITuple makeCopy() {
+            return new TsOnlyTuple(this.partitionCol, this.tsStr);
+        }
+
+        @Override
+        public Object getValue(TblColRef col) {
+            if (col.equals(partitionCol)) {
+                return Tuple.dateToEpicDays(this.tsStr);
+            } else {
+                throw new NotImplementedException();
+            }
+        }
+    }
+
+    @Test
+    public void basicTest() {
+
+        final StorageContext context = new StorageContext();
+        final List<TblColRef> groups = StorageMockUtils.buildGroups();
+        final TblColRef partitionCol = groups.get(0);
+        final List<FunctionDesc> aggregations = StorageMockUtils.buildAggregations();
+        final TupleInfo tupleInfo = StorageMockUtils.newTupleInfo(groups, aggregations);
+
+        SQLDigest sqlDigest = new SQLDigest("default.test_kylin_fact", null, null, Lists.<TblColRef> newArrayList(), groups, Lists.newArrayList(partitionCol), Lists.<TblColRef>newArrayList(), aggregations);
+
+        ITuple aTuple = new TsOnlyTuple(partitionCol, "2011-02-01");
+        ITuple bTuple = new TsOnlyTuple(partitionCol, "2012-02-01");
+        final List<ITuple> allTuples = Lists.newArrayList(aTuple, bTuple);
+
+        //counts for verifying
+        final AtomicInteger underlyingSEHitCount = new AtomicInteger(0);
+        final List<Integer> returnedRowPerSearch = Lists.newArrayList();
+
+        CacheFledgedDynamicStorageEngine dynamicCache = new CacheFledgedDynamicStorageEngine(new IStorageEngine() {
+            @Override
+            public ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo) {
+                Range<Long> tsRagneInQuery = TsConditionExtractor.extractTsCondition(partitionCol, sqlDigest.filter);
+                List<ITuple> ret = Lists.newArrayList();
+                for (ITuple tuple : allTuples) {
+                    if (tsRagneInQuery.contains(Tuple.getTs(tuple, partitionCol))) {
+                        ret.add(tuple);
+                    }
+                }
+
+                underlyingSEHitCount.incrementAndGet();
+                returnedRowPerSearch.add(ret.size());
+
+                return new SimpleTupleIterator(ret.iterator());
+            }
+
+            @Override
+            public boolean isDynamic() {
+                return true;
+            }
+
+            @Override
+            public Range<Long> getVolatilePeriod() {
+                return Ranges.greaterThan(DateFormat.stringToMillis("2011-02-01"));
+            }
+        }, partitionCol);
+
+        sqlDigest.filter = StorageMockUtils.buildTs2010Filter(groups.get(0));
+        ITupleIterator firstIterator = dynamicCache.search(context, sqlDigest, tupleInfo);
+        IdentityHashMap<ITuple, Void> firstResults = new IdentityHashMap<>();
+        while (firstIterator.hasNext()) {
+            firstResults.put(firstIterator.next(), null);
+        }
+        firstIterator.close();
+
+        sqlDigest.filter = StorageMockUtils.buildTs2011Filter(groups.get(0));
+        ITupleIterator secondIterator = dynamicCache.search(context, sqlDigest, tupleInfo);
+        IdentityHashMap<ITuple, Void> secondResults = new IdentityHashMap<>();
+        while (secondIterator.hasNext()) {
+            secondResults.put(secondIterator.next(), null);
+        }
+        secondIterator.close();
+
+        Assert.assertEquals(2, firstResults.size());
+        IdentityUtils.collectionReferenceEquals(firstResults.keySet(), secondResults.keySet());
+        Assert.assertEquals(2, underlyingSEHitCount.get());
+        Assert.assertEquals(new Integer(2), returnedRowPerSearch.get(0));
+        Assert.assertEquals(new Integer(1), returnedRowPerSearch.get(1));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/storage/src/test/java/org/apache/kylin/storage/test/StaticCacheTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/test/StaticCacheTest.java b/storage/src/test/java/org/apache/kylin/storage/test/StaticCacheTest.java
new file mode 100644
index 0000000..9b69830
--- /dev/null
+++ b/storage/src/test/java/org/apache/kylin/storage/test/StaticCacheTest.java
@@ -0,0 +1,95 @@
+package org.apache.kylin.storage.test;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import org.apache.kylin.common.util.IdentityUtils;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.realization.SQLDigest;
+import org.apache.kylin.metadata.tuple.ITuple;
+import org.apache.kylin.metadata.tuple.ITupleIterator;
+import org.apache.kylin.metadata.tuple.SimpleTupleIterator;
+import org.apache.kylin.storage.IStorageEngine;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.cache.CacheFledgedStaticStorageEngine;
+import org.apache.kylin.storage.tuple.Tuple;
+import org.apache.kylin.storage.tuple.TupleInfo;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Created by Hongbin Ma(Binmahone) on 5/14/15.
+ */
+public class StaticCacheTest {
+
+    @Test
+    public void basicTest() {
+
+        final StorageContext context = new StorageContext();
+        final List<TblColRef> groups = StorageMockUtils.buildGroups();
+        final List<FunctionDesc> aggregations = StorageMockUtils.buildAggregations();
+        final TupleFilter filter = StorageMockUtils.buildFilter1(groups.get(0));
+        final SQLDigest sqlDigest = new SQLDigest("default.test_kylin_fact", filter, null, Collections.<TblColRef> emptySet(), groups, Collections.<TblColRef> emptySet(), Collections.<TblColRef> emptySet(), aggregations);
+        final TupleInfo tupleInfo = StorageMockUtils.newTupleInfo(groups, aggregations);
+
+        final List<ITuple> ret = Lists.newArrayList();
+        ret.add(new Tuple(tupleInfo));
+        ret.add(new Tuple(tupleInfo));
+        ret.add(new Tuple(tupleInfo));
+
+        final AtomicInteger underlyingSEHitCount = new AtomicInteger(0);
+
+        CacheFledgedStaticStorageEngine cacheFledgedStaticStorageEngine = new CacheFledgedStaticStorageEngine(new IStorageEngine() {
+            @Override
+            public ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo) {
+                underlyingSEHitCount.incrementAndGet();
+                return new SimpleTupleIterator(ret.iterator());
+            }
+
+            @Override
+            public boolean isDynamic() {
+                return false;
+            }
+
+            @Override
+            public Range<Long> getVolatilePeriod() {
+                return null;
+            }
+        });
+
+        ITupleIterator firstIterator = cacheFledgedStaticStorageEngine.search(context, sqlDigest, tupleInfo);
+        IdentityHashMap<ITuple, Void> firstResults = new IdentityHashMap<>();
+        while (firstIterator.hasNext()) {
+            firstResults.put(firstIterator.next(), null);
+        }
+        firstIterator.close();
+
+        ITupleIterator secondIterator = cacheFledgedStaticStorageEngine.search(context, sqlDigest, tupleInfo);
+        IdentityHashMap<ITuple,Void> secondResults = new IdentityHashMap<>();
+        while(secondIterator.hasNext())
+        {
+            secondResults.put(secondIterator.next(),null);
+        }
+        secondIterator.close();
+
+        ITupleIterator thirdIterator = cacheFledgedStaticStorageEngine.search(context, sqlDigest, tupleInfo);
+        IdentityHashMap<ITuple,Void> thirdResults = new IdentityHashMap<>();
+        while(thirdIterator.hasNext())
+        {
+            thirdResults.put(thirdIterator.next(),null);
+        }
+        thirdIterator.close();
+
+        Assert.assertEquals(3, firstResults.size());
+        IdentityUtils.collectionReferenceEquals(firstResults.keySet(), secondResults.keySet());
+        IdentityUtils.collectionReferenceEquals(thirdResults.keySet(), secondResults.keySet());
+
+        Assert.assertEquals(1, underlyingSEHitCount.get());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/storage/src/test/java/org/apache/kylin/storage/test/StorageMockUtils.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/test/StorageMockUtils.java b/storage/src/test/java/org/apache/kylin/storage/test/StorageMockUtils.java
new file mode 100644
index 0000000..d6a6da1
--- /dev/null
+++ b/storage/src/test/java/org/apache/kylin/storage/test/StorageMockUtils.java
@@ -0,0 +1,132 @@
+package org.apache.kylin.storage.test;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.kylin.metadata.filter.*;
+import org.apache.kylin.metadata.model.*;
+import org.apache.kylin.storage.tuple.TupleInfo;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Created by Hongbin Ma(Binmahone) on 5/14/15.
+ */
+public class StorageMockUtils {
+    public static  TupleInfo newTupleInfo(List<TblColRef> groups, List<FunctionDesc> aggregations) {
+        TupleInfo info = new TupleInfo();
+        int idx = 0;
+
+        for (TblColRef col : groups) {
+            info.setField(col.getName(), col, idx++);
+        }
+
+        TableDesc sourceTable = groups.get(0).getColumnDesc().getTable();
+        for (FunctionDesc func : aggregations) {
+            TblColRef col = new TblColRef(func.newFakeRewriteColumn(sourceTable));
+            info.setField(col.getName(), col, idx++);
+        }
+
+        return info;
+    }
+
+    public static  List<TblColRef> buildGroups() {
+        List<TblColRef> groups = new ArrayList<TblColRef>();
+
+        TableDesc t1 = TableDesc.mockup("DEFAULT.TEST_KYLIN_FACT");
+        ColumnDesc c1 = ColumnDesc.mockup(t1, 2, "CAL_DT", "date");
+        TblColRef cf1 = new TblColRef(c1);
+        groups.add(cf1);
+
+        TableDesc t2 = TableDesc.mockup("DEFAULT.TEST_CATEGORY_GROUPINGS");
+        ColumnDesc c2 = ColumnDesc.mockup(t2, 14, "META_CATEG_NAME", "string");
+        TblColRef cf2 = new TblColRef(c2);
+        groups.add(cf2);
+
+        return groups;
+    }
+
+    public static  List<FunctionDesc> buildAggregations() {
+        List<FunctionDesc> functions = new ArrayList<FunctionDesc>();
+
+        TableDesc t1 = TableDesc.mockup("DEFAULT.TEST_KYLIN_FACT");
+        TblColRef priceCol = new TblColRef(ColumnDesc.mockup(t1, 7, "PRICE", "decimal(19,4)"));
+        TblColRef sellerCol = new TblColRef(ColumnDesc.mockup(t1, 9, "SELLER_ID", "bigint"));
+
+        FunctionDesc f1 = new FunctionDesc();
+        f1.setExpression("SUM");
+        ParameterDesc p1 = new ParameterDesc();
+        p1.setType("column");
+        p1.setValue("PRICE");
+        p1.setColRefs(ImmutableList.of(priceCol));
+        f1.setParameter(p1);
+        functions.add(f1);
+
+        FunctionDesc f2 = new FunctionDesc();
+        f2.setExpression("COUNT_DISTINCT");
+        ParameterDesc p2 = new ParameterDesc();
+        p2.setType("column");
+        p2.setValue("SELLER_ID");
+        p2.setColRefs(ImmutableList.of(sellerCol));
+        f2.setParameter(p2);
+        functions.add(f2);
+
+        return functions;
+    }
+
+
+    public static  CompareTupleFilter buildTs2010Filter(TblColRef column) {
+        CompareTupleFilter compareFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.GT);
+        ColumnTupleFilter columnFilter1 = new ColumnTupleFilter(column);
+        compareFilter.addChild(columnFilter1);
+        ConstantTupleFilter constantFilter1 = new ConstantTupleFilter("2010-01-01");
+        compareFilter.addChild(constantFilter1);
+        return compareFilter;
+    }
+
+    public static  CompareTupleFilter buildTs2011Filter(TblColRef column) {
+        CompareTupleFilter compareFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.GT);
+        ColumnTupleFilter columnFilter1 = new ColumnTupleFilter(column);
+        compareFilter.addChild(columnFilter1);
+        ConstantTupleFilter constantFilter1 = new ConstantTupleFilter("2011-01-01");
+        compareFilter.addChild(constantFilter1);
+        return compareFilter;
+    }
+
+    public static  CompareTupleFilter buildFilter1(TblColRef column) {
+        CompareTupleFilter compareFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.EQ);
+        ColumnTupleFilter columnFilter1 = new ColumnTupleFilter(column);
+        compareFilter.addChild(columnFilter1);
+        ConstantTupleFilter constantFilter1 = new ConstantTupleFilter("2012-05-23");
+        compareFilter.addChild(constantFilter1);
+        return compareFilter;
+    }
+
+    public static  CompareTupleFilter buildFilter2(TblColRef column) {
+        CompareTupleFilter compareFilter = new CompareTupleFilter(TupleFilter.FilterOperatorEnum.EQ);
+        ColumnTupleFilter columnFilter2 = new ColumnTupleFilter(column);
+        compareFilter.addChild(columnFilter2);
+        ConstantTupleFilter constantFilter2 = new ConstantTupleFilter("ClothinShoes & Accessories");
+        compareFilter.addChild(constantFilter2);
+        return compareFilter;
+    }
+
+    @SuppressWarnings("unused")
+    public static  TupleFilter buildAndFilter(List<TblColRef> columns) {
+        CompareTupleFilter compareFilter1 = buildFilter1(columns.get(0));
+        CompareTupleFilter compareFilter2 = buildFilter2(columns.get(1));
+        LogicalTupleFilter andFilter = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.AND);
+        andFilter.addChild(compareFilter1);
+        andFilter.addChild(compareFilter2);
+        return andFilter;
+    }
+
+    @SuppressWarnings("unused")
+    public static  TupleFilter buildOrFilter(List<TblColRef> columns) {
+        CompareTupleFilter compareFilter1 = buildFilter1(columns.get(0));
+        CompareTupleFilter compareFilter2 = buildFilter2(columns.get(1));
+        LogicalTupleFilter logicFilter = new LogicalTupleFilter(TupleFilter.FilterOperatorEnum.OR);
+        logicFilter.addChild(compareFilter1);
+        logicFilter.addChild(compareFilter2);
+        return logicFilter;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0e42b2df/storage/src/test/java/org/apache/kylin/storage/test/StorageTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/test/StorageTest.java b/storage/src/test/java/org/apache/kylin/storage/test/StorageTest.java
index fece398..f9a6212 100644
--- a/storage/src/test/java/org/apache/kylin/storage/test/StorageTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/test/StorageTest.java
@@ -18,14 +18,13 @@
 
 package org.apache.kylin.storage.test;
 
-import com.google.common.collect.ImmutableList;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.metadata.filter.*;
-import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
-import org.apache.kylin.metadata.model.*;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.SQLDigest;
 import org.apache.kylin.metadata.tuple.ITuple;
 import org.apache.kylin.metadata.tuple.ITupleIterator;
@@ -33,10 +32,8 @@ import org.apache.kylin.storage.IStorageEngine;
 import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.storage.StorageEngineFactory;
 import org.apache.kylin.storage.hbase.ScanOutOfLimitException;
-import org.apache.kylin.storage.tuple.TupleInfo;
 import org.junit.*;
 
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
@@ -78,17 +75,17 @@ public class StorageTest extends HBaseMetadataTestCase {
     @Ignore
     public void testScanOutOfLimit() {
         context.setThreshold(1);
-        List<TblColRef> groups = buildGroups();
-        List<FunctionDesc> aggregations = buildAggregations();
+        List<TblColRef> groups = StorageMockUtils.buildGroups();
+        List<FunctionDesc> aggregations = StorageMockUtils.buildAggregations();
 
         search(groups, aggregations, null, context);
     }
 
     @Test
     public void test01() {
-        List<TblColRef> groups = buildGroups();
-        List<FunctionDesc> aggregations = buildAggregations();
-        TupleFilter filter = buildFilter1(groups.get(0));
+        List<TblColRef> groups = StorageMockUtils.buildGroups();
+        List<FunctionDesc> aggregations = StorageMockUtils.buildAggregations();
+        TupleFilter filter = StorageMockUtils.buildFilter1(groups.get(0));
 
         int count = search(groups, aggregations, filter, context);
         assertTrue(count > 0);
@@ -139,7 +136,7 @@ public class StorageTest extends HBaseMetadataTestCase {
         ITupleIterator iterator = null;
         try {
             SQLDigest sqlDigest = new SQLDigest("default.test_kylin_fact", filter, null, Collections.<TblColRef> emptySet(), groups, Collections.<TblColRef> emptySet(), Collections.<TblColRef> emptySet(), aggregations);
-            iterator = storageEngine.search(context, sqlDigest, newTupleInfo(groups, aggregations));
+            iterator = storageEngine.search(context, sqlDigest, StorageMockUtils.newTupleInfo(groups, aggregations));
             while (iterator.hasNext()) {
                 ITuple tuple = iterator.next();
                 System.out.println("Tuple = " + tuple);
@@ -154,102 +151,4 @@ public class StorageTest extends HBaseMetadataTestCase {
         return count;
     }
 
-    private TupleInfo newTupleInfo(List<TblColRef> groups, List<FunctionDesc> aggregations) {
-        TupleInfo info = new TupleInfo();
-        int idx = 0;
-        
-        for (TblColRef col : groups) {
-            info.setField(col.getName(), col, idx++);
-        }
-        
-        TableDesc sourceTable = groups.get(0).getColumnDesc().getTable();
-        for (FunctionDesc func : aggregations) {
-            TblColRef col = new TblColRef(func.newFakeRewriteColumn(sourceTable));
-            info.setField(col.getName(), col, idx++);
-        }
-        
-        return info;
-    }
-
-    private List<TblColRef> buildGroups() {
-        List<TblColRef> groups = new ArrayList<TblColRef>();
-        
-        TableDesc t1 = TableDesc.mockup("DEFAULT.TEST_KYLIN_FACT");
-        ColumnDesc c1 = ColumnDesc.mockup(t1, 2, "CAL_DT", "string");
-        TblColRef cf1 = new TblColRef(c1);
-        groups.add(cf1);
-
-        TableDesc t2 = TableDesc.mockup("DEFAULT.TEST_CATEGORY_GROUPINGS");
-        ColumnDesc c2 = ColumnDesc.mockup(t2, 14, "META_CATEG_NAME", "string");
-        TblColRef cf2 = new TblColRef(c2);
-        groups.add(cf2);
-
-        return groups;
-    }
-
-    private List<FunctionDesc> buildAggregations() {
-        List<FunctionDesc> functions = new ArrayList<FunctionDesc>();
-
-        TableDesc t1 = TableDesc.mockup("DEFAULT.TEST_KYLIN_FACT");
-        TblColRef priceCol = new TblColRef(ColumnDesc.mockup(t1, 7, "PRICE", "decimal(19,4)"));
-        TblColRef sellerCol = new TblColRef(ColumnDesc.mockup(t1, 9, "SELLER_ID", "bigint"));
-        
-        FunctionDesc f1 = new FunctionDesc();
-        f1.setExpression("SUM");
-        ParameterDesc p1 = new ParameterDesc();
-        p1.setType("column");
-        p1.setValue("PRICE");
-        p1.setColRefs(ImmutableList.of(priceCol));
-        f1.setParameter(p1);
-        functions.add(f1);
-
-        FunctionDesc f2 = new FunctionDesc();
-        f2.setExpression("COUNT_DISTINCT");
-        ParameterDesc p2 = new ParameterDesc();
-        p2.setType("column");
-        p2.setValue("SELLER_ID");
-        p2.setColRefs(ImmutableList.of(sellerCol));
-        f2.setParameter(p2);
-        functions.add(f2);
-
-        return functions;
-    }
-
-    private CompareTupleFilter buildFilter1(TblColRef column) {
-        CompareTupleFilter compareFilter = new CompareTupleFilter(FilterOperatorEnum.EQ);
-        ColumnTupleFilter columnFilter1 = new ColumnTupleFilter(column);
-        compareFilter.addChild(columnFilter1);
-        ConstantTupleFilter constantFilter1 = new ConstantTupleFilter("2012-05-23");
-        compareFilter.addChild(constantFilter1);
-        return compareFilter;
-    }
-
-    private CompareTupleFilter buildFilter2(TblColRef column) {
-        CompareTupleFilter compareFilter = new CompareTupleFilter(FilterOperatorEnum.EQ);
-        ColumnTupleFilter columnFilter2 = new ColumnTupleFilter(column);
-        compareFilter.addChild(columnFilter2);
-        ConstantTupleFilter constantFilter2 = new ConstantTupleFilter("ClothinShoes & Accessories");
-        compareFilter.addChild(constantFilter2);
-        return compareFilter;
-    }
-
-    @SuppressWarnings("unused")
-    private TupleFilter buildAndFilter(List<TblColRef> columns) {
-        CompareTupleFilter compareFilter1 = buildFilter1(columns.get(0));
-        CompareTupleFilter compareFilter2 = buildFilter2(columns.get(1));
-        LogicalTupleFilter andFilter = new LogicalTupleFilter(FilterOperatorEnum.AND);
-        andFilter.addChild(compareFilter1);
-        andFilter.addChild(compareFilter2);
-        return andFilter;
-    }
-
-    @SuppressWarnings("unused")
-    private TupleFilter buildOrFilter(List<TblColRef> columns) {
-        CompareTupleFilter compareFilter1 = buildFilter1(columns.get(0));
-        CompareTupleFilter compareFilter2 = buildFilter2(columns.get(1));
-        LogicalTupleFilter logicFilter = new LogicalTupleFilter(FilterOperatorEnum.OR);
-        logicFilter.addChild(compareFilter1);
-        logicFilter.addChild(compareFilter2);
-        return logicFilter;
-    }
 }