You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by su...@apache.org on 2016/06/04 06:41:36 UTC

[1/6] kylin git commit: KYLIN-1719 Add config in scan request to control compress the query result or not

Repository: kylin
Updated Branches:
  refs/heads/KYLIN-1705-B2 [created] 3032bbbe6


KYLIN-1719 Add config in scan request to control compress the query result or not


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

Branch: refs/heads/KYLIN-1705-B2
Commit: 773ca515ea6d6c8ebe00597090a389fb3975f466
Parents: 258e765
Author: sunyerui <su...@gmail.com>
Authored: Wed Jun 1 22:42:39 2016 +0800
Committer: sunyerui <su...@gmail.com>
Committed: Fri Jun 3 18:50:22 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/cube/model/validation/rule/DictionaryRule.java    | 1 +
 .../apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java   | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/773ca515/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java
new file mode 100644
index 0000000..7ce263d
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java
@@ -0,0 +1 @@
+package org.apache.kylin.cube.model.validation.rule;

/**
 * Created by sunyerui on 16/6/1.
 */
public class DictionaryRule {
}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/773ca515/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
index f071a1b..ac81f55 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
@@ -321,7 +321,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
 
         final AtomicInteger totalScannedCount = new AtomicInteger(0);
         final ExpectedSizeIterator epResultItr = new ExpectedSizeIterator(shardNum);
-        final boolean compressionResult = cubeSeg.getCubeDesc().getConfig().getCompressionResult();
+        final boolean compressionResult = KylinConfig.getInstanceFromEnv().getCompressionResult();
         final CubeVisitProtos.CubeVisitRequest.Builder builder = CubeVisitProtos.CubeVisitRequest.newBuilder();
         builder.setGtScanRequest(scanRequestByteString).setHbaseRawScan(rawScanByteString);
         for (IntList intList : hbaseColumnsToGTIntList) {


[3/6] kylin git commit: KYLIN-1705 Global (and more scalable) dictionary

Posted by su...@apache.org.
KYLIN-1705 Global (and more scalable) dictionary


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

Branch: refs/heads/KYLIN-1705-B2
Commit: d1a9bab640a7ad422f3f8ef091243033161417ab
Parents: 773ca51
Author: sunyerui <su...@gmail.com>
Authored: Fri May 27 00:50:14 2016 +0800
Committer: sunyerui <su...@gmail.com>
Committed: Fri Jun 3 18:59:34 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/KylinConfigBase.java    |   16 +
 .../common/persistence/ComparableWritable.java  |   25 +
 .../apache/kylin/cube/model/DictionaryDesc.java |    9 +
 .../model/validation/CubeMetadataValidator.java |    3 +-
 .../model/validation/rule/DictionaryRule.java   |   87 +-
 .../validation/rule/DictionaryRuleTest.java     |  108 ++
 core-dictionary/pom.xml                         |    5 +
 .../apache/kylin/dict/AppendTrieDictionary.java | 1125 ++++++++++++++++++
 .../org/apache/kylin/dict/CachedTreeMap.java    |  348 ++++++
 .../apache/kylin/dict/DictionaryGenerator.java  |   24 +-
 .../apache/kylin/dict/DictionaryManager.java    |    2 +-
 .../kylin/dict/GlobalDictionaryBuilder.java     |   85 ++
 .../apache/kylin/dict/IDictionaryBuilder.java   |    2 +-
 .../kylin/dict/AppendTrieDictionaryTest.java    |  229 ++++
 .../engine/mr/steps/MergeCuboidMapper.java      |   18 +-
 15 files changed, 2066 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 0199378..af0c00b 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -703,4 +703,20 @@ abstract public class KylinConfigBase implements Serializable {
     public Map<String, String> getUDFs() {
         return getPropertiesByPrefix("kylin.query.udf.");
     }
+
+    public int getAppendDictEntrySize() {
+        return Integer.parseInt(getOptional("kylin.dict.append.entry.size", "10000000"));
+    }
+
+    public void setAppendDictEntrySize(int entrySize) {
+        setProperty("kylin.dict.append.entry.size", String.valueOf(entrySize));
+    }
+
+    public int getAppendDictCacheSize() {
+        return Integer.parseInt(getOptional("kylin.dict.append.cache.size", "20"));
+    }
+
+    public void setAppendDictCacheSize(int cacheSize) {
+        setProperty("kylin.dict.append.cache.size", String.valueOf(cacheSize));
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-common/src/main/java/org/apache/kylin/common/persistence/ComparableWritable.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/ComparableWritable.java b/core-common/src/main/java/org/apache/kylin/common/persistence/ComparableWritable.java
new file mode 100644
index 0000000..5dae9cb
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/persistence/ComparableWritable.java
@@ -0,0 +1,25 @@
+/*
+ * 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.common.persistence;
+
+/**
+ * Created by sunyerui on 16/5/11.
+ */
+public interface ComparableWritable extends Comparable, Writable {
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java
index 47bba33..3682e41 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java
@@ -65,4 +65,13 @@ public class DictionaryDesc {
     public String getBuilderClass() {
         return builderClass;
     }
+
+    // for test
+    public static DictionaryDesc create(String column, String reuseColumn, String builderClass) {
+        DictionaryDesc desc = new DictionaryDesc();
+        desc.column = column;
+        desc.reuseColumn = reuseColumn;
+        desc.builderClass = builderClass;
+        return desc;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java
index 7d7710c..d2f84f3 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/CubeMetadataValidator.java
@@ -20,6 +20,7 @@ package org.apache.kylin.cube.model.validation;
 
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.validation.rule.AggregationGroupRule;
+import org.apache.kylin.cube.model.validation.rule.DictionaryRule;
 import org.apache.kylin.cube.model.validation.rule.FunctionRule;
 import org.apache.kylin.cube.model.validation.rule.RowKeyAttrRule;
 
@@ -31,7 +32,7 @@ import org.apache.kylin.cube.model.validation.rule.RowKeyAttrRule;
  */
 public class CubeMetadataValidator {
     @SuppressWarnings("unchecked")
-    private IValidatorRule<CubeDesc>[] rules = new IValidatorRule[] { new FunctionRule(), new AggregationGroupRule(), new RowKeyAttrRule() };
+    private IValidatorRule<CubeDesc>[] rules = new IValidatorRule[] { new FunctionRule(), new AggregationGroupRule(), new RowKeyAttrRule(), new DictionaryRule()};
 
     public ValidateContext validate(CubeDesc cube) {
         return validate(cube, false);

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java
index 7ce263d..effe30c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java
@@ -1 +1,86 @@
-package org.apache.kylin.cube.model.validation.rule;

/**
 * Created by sunyerui on 16/6/1.
 */
public class DictionaryRule {
}
\ No newline at end of file
+/*
+ * 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.model.validation.rule;
+
+import org.apache.kylin.cube.model.*;
+import org.apache.kylin.cube.model.validation.IValidatorRule;
+import org.apache.kylin.cube.model.validation.ResultLevel;
+import org.apache.kylin.cube.model.validation.ValidateContext;
+import org.apache.kylin.dict.AppendTrieDictionary;
+import org.apache.kylin.dict.GlobalDictionaryBuilder;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import java.util.HashMap;
+import java.util.List;
+
+/**
+ * Created by sunyerui on 16/6/1.
+ */
+public class DictionaryRule implements IValidatorRule<CubeDesc> {
+
+    @Override
+    public void validate(CubeDesc cubeDesc, ValidateContext context) {
+        List<DictionaryDesc> dictDescs = cubeDesc.getDictionaries();
+        if (dictDescs == null || dictDescs.isEmpty()) {
+            return;
+        }
+
+        HashMap<TblColRef, String> colToBuilderMap = new HashMap<>();
+        HashMap<TblColRef, TblColRef> colToReuseColMap = new HashMap<>();
+        for (DictionaryDesc dictDesc : dictDescs) {
+            // Make sure the same column associate with same builder class, check the reuse column by default
+            TblColRef dictCol = dictDesc.getResuseColumnRef();
+            if (dictCol == null) {
+                dictCol = dictDesc.getColumnRef();
+            }
+            if (dictCol == null) {
+                context.addResult(ResultLevel.ERROR, "Some column in dictionaries not found");
+                return;
+            }
+            String builder = dictDesc.getBuilderClass();
+            String oldBuilder = colToBuilderMap.put(dictCol, builder);
+            if (oldBuilder != null && !oldBuilder.equals(builder)) {
+                context.addResult(ResultLevel.ERROR, "Column " + dictCol + " has inconsistent builders " + builder + " and " + oldBuilder);
+                return;
+            }
+
+            // Make sure one column only reuse another one column
+            if (dictDesc.getResuseColumnRef() != null) {
+                TblColRef oldReuseCol = colToReuseColMap.put(dictDesc.getColumnRef(), dictDesc.getResuseColumnRef());
+                if (oldReuseCol != null && !dictDesc.getResuseColumnRef().equals(oldReuseCol)) {
+                    context.addResult(ResultLevel.ERROR, "Column " + dictDesc.getColumnRef() + " reuse inconsistent column " + dictDesc.getResuseColumnRef() + " and " + oldReuseCol);
+                    return;
+                }
+            }
+        }
+
+        // Make sure one column do not use GlobalDictBuilder and DimensionDictEncoding together
+        RowKeyColDesc[] rowKeyColDescs = cubeDesc.getRowkey().getRowKeyColumns();
+        for (RowKeyColDesc desc : rowKeyColDescs) {
+            if (desc.isUsingDictionary()) {
+                String builder = colToBuilderMap.get(desc.getColRef());
+                if (builder != null && builder.equals(GlobalDictionaryBuilder.class.getName())) {
+                    context.addResult(ResultLevel.ERROR, "Column " + desc.getColRef() + " used as dimension and conflict with GlobalDictBuilder");
+                    return;
+                }
+            }
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
new file mode 100644
index 0000000..ba58d40
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.model.validation.rule;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.DictionaryDesc;
+import org.apache.kylin.cube.model.validation.ValidateContext;
+import org.apache.kylin.dict.GlobalDictionaryBuilder;
+import org.apache.kylin.metadata.MetadataManager;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import static org.junit.Assert.*;
+
+/**
+ * Created by sunyerui on 16/6/1.
+ */
+public class DictionaryRuleTest extends LocalFileMetadataTestCase {
+    private static KylinConfig config;
+    private static MetadataManager metadataManager;
+
+    @Before
+    public void setUp() throws Exception {
+        this.createTestMetadata();
+        config = KylinConfig.getInstanceFromEnv();
+        metadataManager = MetadataManager.getInstance(config);
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testGoodDesc() throws IOException {
+        DictionaryRule rule = new DictionaryRule();
+
+        for (File f : new File(LocalFileMetadataTestCase.LOCALMETA_TEST_DATA + "/cube_desc/").listFiles()) {
+            CubeDesc desc = JsonUtil.readValue(new FileInputStream(f), CubeDesc.class);
+            desc.init(config, metadataManager.getAllTablesMap());
+            ValidateContext vContext = new ValidateContext();
+            rule.validate(desc, vContext);
+            vContext.print(System.out);
+            assertTrue(vContext.getResults().length == 0);
+        }
+    }
+
+    @Test
+    public void testBadDesc() throws IOException {
+        testBadDictionaryDesc("Column DEFAULT.TEST_KYLIN_FACT.SELLER_ID has inconsistent builders " +
+                "FakeBuilderClass and org.apache.kylin.dict.GlobalDictionaryBuilder",
+            DictionaryDesc.create("SELLER_ID", null, "FakeBuilderClass"));
+    }
+
+    @Test
+    public void testBadDesc2() throws IOException {
+        testBadDictionaryDesc("Column DEFAULT.TEST_KYLIN_FACT.SELLER_ID has inconsistent builders " +
+                        "FakeBuilderClass and org.apache.kylin.dict.GlobalDictionaryBuilder",
+                DictionaryDesc.create("lstg_site_id", "SELLER_ID", "FakeBuilderClass"));
+    }
+
+    @Test
+    public void testBadDesc3() throws IOException {
+        testBadDictionaryDesc("Column DEFAULT.TEST_KYLIN_FACT.LSTG_SITE_ID used as dimension and conflict with GlobalDictBuilder",
+                DictionaryDesc.create("lstg_site_id", null, GlobalDictionaryBuilder.class.getName()));
+    }
+
+    private void testBadDictionaryDesc(String expectMessage, DictionaryDesc... descs) throws IOException {
+        DictionaryRule rule = new DictionaryRule();
+        File f = new File(LocalFileMetadataTestCase.LOCALMETA_TEST_DATA + "/cube_desc/test_kylin_cube_without_slr_left_join_desc.json");
+        CubeDesc desc = JsonUtil.readValue(new FileInputStream(f), CubeDesc.class);
+
+        for (DictionaryDesc dictDesc: descs) {
+            desc.getDictionaries().add(dictDesc);
+        }
+
+        desc.init(config, metadataManager.getAllTablesMap());
+        ValidateContext vContext = new ValidateContext();
+        rule.validate(desc, vContext);
+        vContext.print(System.out);
+        assertTrue(vContext.getResults().length >= 1);
+        assertEquals(expectMessage, vContext.getResults()[0].getMessage());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-dictionary/pom.xml
----------------------------------------------------------------------
diff --git a/core-dictionary/pom.xml b/core-dictionary/pom.xml
index 17319e0..ddcbf53 100644
--- a/core-dictionary/pom.xml
+++ b/core-dictionary/pom.xml
@@ -41,6 +41,11 @@
             <artifactId>kylin-core-metadata</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>compile</scope>
+        </dependency>
 
         <!-- Env & Test -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
new file mode 100644
index 0000000..a51a798
--- /dev/null
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
@@ -0,0 +1,1125 @@
+/*
+ * 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.dict;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.ComparableWritable;
+import org.apache.kylin.common.persistence.Writable;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.common.util.Dictionary;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.UnsupportedEncodingException;
+import java.lang.ref.SoftReference;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.IdentityHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A dictionary based on Trie data structure that maps enumerations of byte[] to
+ * int IDs, used for global dictionary.
+ *
+ * Trie data is split into sub trees, called {@link DictSlice}, and stored in a {@link CachedTreeMap} with a configurable cache size.
+ * 
+ * With Trie the memory footprint of the mapping is kinda minimized at the cost
+ * CPU, if compared to HashMap of ID Arrays. Performance test shows Trie is
+ * roughly 10 times slower, so there's a cache layer overlays on top of Trie and
+ * gracefully fall back to Trie using a weak reference.
+ * 
+ * The implementation is NOT thread-safe for now.
+ *
+ * TODO making it thread-safe
+ * 
+ * @author sunyerui
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class AppendTrieDictionary<T> extends Dictionary<T> {
+
+    public static final byte[] HEAD_MAGIC = new byte[] { 0x41, 0x70, 0x70, 0x65, 0x63, 0x64, 0x54, 0x72, 0x69, 0x65, 0x44, 0x69, 0x63, 0x74 }; // "AppendTrieDict"
+    public static final int HEAD_SIZE_I = HEAD_MAGIC.length;
+
+    public static final int BIT_IS_LAST_CHILD = 0x80;
+    public static final int BIT_IS_END_OF_VALUE = 0x40;
+
+    private static final Logger logger = LoggerFactory.getLogger(AppendTrieDictionary.class);
+
+    transient private String baseDir;
+    transient private int baseId;
+    transient private int maxId;
+    transient private int maxValueLength;
+    transient private int nValues;
+    transient private BytesConverter<T> bytesConverter;
+
+    private TreeMap<DictSliceKey, DictSlice> dictSliceMap;
+
+    transient private boolean enableValueCache = true;
+    transient private SoftReference<HashMap> valueToIdCache;
+
+    // Constructor both for build and deserialize
+    public AppendTrieDictionary() {
+        if (enableValueCache) {
+            valueToIdCache = new SoftReference<>(new HashMap());
+        }
+    }
+
+    public void update(String baseDir, int baseId, int maxId, int maxValueLength, int nValues, BytesConverter bytesConverter, byte[] dictMapBytes) throws IOException {
+        ByteArrayInputStream buf = new ByteArrayInputStream(dictMapBytes);
+        DataInputStream input = new DataInputStream(buf);
+        update(baseDir, baseId, maxId, maxValueLength, nValues, bytesConverter, input);
+    }
+
+    public void update(String baseDir, int baseId, int maxId, int maxValueLength, int nValues, BytesConverter bytesConverter, DataInput input) throws IOException {
+        this.baseDir = baseDir;
+        this.baseId = baseId;
+        this.maxId = maxId;
+        this.maxValueLength = maxValueLength;
+        this.nValues = nValues;
+        this.bytesConverter = bytesConverter;
+
+        int cacheSize = KylinConfig.getInstanceFromEnv().getAppendDictCacheSize();
+        if (dictSliceMap == null) {
+            dictSliceMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder().maxSize(cacheSize)
+                    .baseDir(baseDir).persistent(true).immutable(true).keyClazz(DictSliceKey.class).valueClazz(DictSlice.class).build();
+        }
+        dictSliceMap.clear();
+        ((Writable)dictSliceMap).readFields(input);
+    }
+
+
+    public byte[] writeDictMap() throws IOException {
+        ByteArrayOutputStream buf = new ByteArrayOutputStream();
+        DataOutputStream out = new DataOutputStream(buf);
+        ((Writable)dictSliceMap).write(out);
+        byte[] dictMapBytes = buf.toByteArray();
+        buf.close();
+        out.close();
+
+        return dictMapBytes;
+    }
+
+    public static class DictSliceKey implements ComparableWritable {
+        byte[] key;
+
+        public static DictSliceKey wrap(byte[] key) {
+            DictSliceKey dictKey = new DictSliceKey();
+            dictKey.key = key;
+            return dictKey;
+        }
+
+        @Override
+        public String toString() {
+            return Bytes.toStringBinary(key);
+        }
+
+        @Override
+        public int hashCode() {
+            return Arrays.hashCode(key);
+        }
+
+        @Override
+        public int compareTo(Object o) {
+            if (!(o instanceof DictSliceKey)) {
+                return -1;
+            }
+            DictSliceKey other = (DictSliceKey)o;
+            return Bytes.compareTo(key, other.key);
+        }
+
+        @Override
+        public void write(DataOutput out) throws IOException {
+            out.writeInt(key.length);
+            out.write(key);
+        }
+
+        @Override
+        public void readFields(DataInput in) throws IOException {
+            key = new byte[in.readInt()];
+            in.readFully(key);
+        }
+    }
+
+    public static class DictSlice<T> implements Writable {
+        public DictSlice() {}
+
+        public DictSlice(byte[] trieBytes) {
+            init(trieBytes);
+        }
+
+        private byte[] trieBytes;
+
+        // non-persistent part
+        transient private int headSize;
+        @SuppressWarnings("unused")
+        transient private int bodyLen;
+        transient private int sizeChildOffset;
+
+        transient private int nValues;
+        transient private int sizeOfId;
+        transient private int childOffsetMask;
+        transient private int firstByteOffset;
+
+        private void init(byte[] trieBytes) {
+            this.trieBytes = trieBytes;
+            if (BytesUtil.compareBytes(HEAD_MAGIC, 0, trieBytes, 0, HEAD_MAGIC.length) != 0)
+                throw new IllegalArgumentException("Wrong file type (magic does not match)");
+
+            try {
+                DataInputStream headIn = new DataInputStream(new ByteArrayInputStream(trieBytes, HEAD_SIZE_I, trieBytes.length - HEAD_SIZE_I));
+                this.headSize = headIn.readShort();
+                this.bodyLen = headIn.readInt();
+                this.nValues = headIn.readInt();
+                this.sizeChildOffset = headIn.read();
+                this.sizeOfId = headIn.read();
+
+                this.childOffsetMask = ~((BIT_IS_LAST_CHILD | BIT_IS_END_OF_VALUE) << ((sizeChildOffset - 1) * 8));
+                this.firstByteOffset = sizeChildOffset + 1; // the offset from begin of node to its first value byte
+            } catch (Exception e) {
+                if (e instanceof RuntimeException)
+                    throw (RuntimeException) e;
+                else
+                    throw new RuntimeException(e);
+            }
+        }
+
+        public byte[] getFirstValue() {
+            int nodeOffset = headSize;
+            ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+            while (true) {
+                int valueLen = BytesUtil.readUnsigned(trieBytes, nodeOffset + firstByteOffset - 1, 1);
+                bytes.write(trieBytes, nodeOffset + firstByteOffset, valueLen);
+                if (checkFlag(nodeOffset, BIT_IS_END_OF_VALUE)) {
+                    break;
+                }
+                nodeOffset = headSize + (BytesUtil.readUnsigned(trieBytes, nodeOffset, sizeChildOffset) & childOffsetMask);
+                if (nodeOffset == headSize) {
+                    break;
+                }
+            }
+            return bytes.toByteArray();
+        }
+
+        /**
+         * returns a code point from [0, nValues), preserving order of value
+         *
+         * @param n
+         *            -- the offset of current node
+         * @param inp
+         *            -- input value bytes to lookup
+         * @param o
+         *            -- offset in the input value bytes matched so far
+         * @param inpEnd
+         *            -- end of input
+         * @param roundingFlag
+         *            -- =0: return -1 if not found
+         *            -- <0: return closest smaller if not found, return -1
+         *            -- >0: return closest bigger if not found, return nValues
+         */
+        private int lookupSeqNoFromValue(int n, byte[] inp, int o, int inpEnd, int roundingFlag) {
+            while (true) {
+                // match the current node
+                int p = n + firstByteOffset; // start of node's value
+                int end = p + BytesUtil.readUnsigned(trieBytes, p - 1, 1); // end of node's value
+                for (; p < end && o < inpEnd; p++, o++) { // note matching start from [0]
+                    if (trieBytes[p] != inp[o]) {
+                        return -1; // mismatch
+                    }
+                }
+
+                // node completely matched, is input all consumed?
+                boolean isEndOfValue = checkFlag(n, BIT_IS_END_OF_VALUE);
+                if (o == inpEnd) {
+                    return p == end && isEndOfValue ? BytesUtil.readUnsigned(trieBytes, end, sizeOfId) : -1;
+                }
+
+                // find a child to continue
+                int c = headSize + (BytesUtil.readUnsigned(trieBytes, n, sizeChildOffset) & childOffsetMask);
+                if (c == headSize) // has no children
+                    return -1;
+                byte inpByte = inp[o];
+                int comp;
+                while (true) {
+                    p = c + firstByteOffset;
+                    comp = BytesUtil.compareByteUnsigned(trieBytes[p], inpByte);
+                    if (comp == 0) { // continue in the matching child, reset n and loop again
+                        n = c;
+                        break;
+                    } else if (comp < 0) { // try next child
+                        if (checkFlag(c, BIT_IS_LAST_CHILD))
+                            return -1;
+                        c = p + BytesUtil.readUnsigned(trieBytes, p - 1, 1) + (checkFlag(c, BIT_IS_END_OF_VALUE) ? sizeOfId : 0);
+                    } else { // children are ordered by their first value byte
+                        return -1;
+                    }
+                }
+            }
+        }
+
+        private boolean checkFlag(int offset, int bit) {
+            return (trieBytes[offset] & bit) > 0;
+        }
+
+        public int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
+            int id = lookupSeqNoFromValue(headSize, value, offset, offset + len, roundingFlag);
+            return id;
+        }
+
+        private DictNode rebuildTrieTree() {
+           return rebuildTrieTreeR(headSize, null);
+        }
+
+        private DictNode rebuildTrieTreeR(int n, DictNode parent) {
+            DictNode root = null;
+            while (true) {
+                int p = n + firstByteOffset;
+                int childOffset = BytesUtil.readUnsigned(trieBytes, n, sizeChildOffset) & childOffsetMask;
+                int parLen = BytesUtil.readUnsigned(trieBytes, p - 1, 1);
+                boolean isEndOfValue = checkFlag(n, BIT_IS_END_OF_VALUE);
+
+                byte[] value = new byte[parLen];
+                System.arraycopy(trieBytes, p, value, 0, parLen);
+
+                DictNode node = new DictNode(value, isEndOfValue);
+                if (isEndOfValue) {
+                    int id = BytesUtil.readUnsigned(trieBytes, p + parLen, sizeOfId);
+                    node.id = id;
+                }
+
+                if (parent == null) {
+                    root = node;
+                } else {
+                    parent.addChild(node);
+                }
+
+                if (childOffset != 0) {
+                    rebuildTrieTreeR(childOffset + headSize, node);
+                }
+
+                if (checkFlag(n, BIT_IS_LAST_CHILD)) {
+                    break;
+                } else {
+                    n += firstByteOffset + parLen + (isEndOfValue ? sizeOfId : 0);
+                }
+            }
+            return root;
+        }
+
+        public void write(DataOutput out) throws IOException {
+            out.write(trieBytes);
+        }
+
+        public void readFields(DataInput in) throws IOException {
+            byte[] headPartial = new byte[HEAD_MAGIC.length + Short.SIZE/Byte.SIZE + Integer.SIZE/Byte.SIZE];
+            in.readFully(headPartial);
+
+            if (BytesUtil.compareBytes(HEAD_MAGIC, 0, headPartial, 0, HEAD_MAGIC.length) != 0)
+                throw new IllegalArgumentException("Wrong file type (magic does not match)");
+
+            DataInputStream headIn = new DataInputStream( //
+                    new ByteArrayInputStream(headPartial, HEAD_SIZE_I, headPartial.length - HEAD_SIZE_I));
+            int headSize = headIn.readShort();
+            int bodyLen = headIn.readInt();
+            headIn.close();
+
+            byte[] all = new byte[headSize + bodyLen];
+            System.arraycopy(headPartial, 0, all, 0, headPartial.length);
+            in.readFully(all, headPartial.length, all.length - headPartial.length);
+
+            init(all);
+        }
+
+        public static DictNode rebuildNodeByDeserialize(DataInput in) throws IOException {
+            DictSlice slice = new DictSlice();
+            slice.readFields(in);
+            return slice.rebuildTrieTree();
+        }
+
+        @Override
+        public String toString() {
+            return String.format("DictSlice[firstValue=%s, values=%d, bytes=%d]", Bytes.toStringBinary(getFirstValue()), nValues, bodyLen);
+        }
+
+        @Override
+        public int hashCode() {
+            return Arrays.hashCode(trieBytes);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if ((o instanceof AppendTrieDictionary.DictSlice) == false) {
+                logger.info("Equals return false because it's not DictInfo");
+                return false;
+            }
+            DictSlice that = (DictSlice) o;
+            return Arrays.equals(this.trieBytes, that.trieBytes);
+        }
+    }
+
+    public static class DictNode implements Writable {
+        public byte[] part;
+        public int id = -1;
+        public boolean isEndOfValue;
+        public ArrayList<DictNode> children = new ArrayList<>();
+
+        public int nValuesBeneath;
+        public DictNode parent;
+        public int childrenCount = 1;
+
+        public DictNode() {}
+
+        public void clone(DictNode o) {
+            this.part = o.part;
+            this.id = o.id;
+            this.isEndOfValue = o.isEndOfValue;
+            this.children = o.children;
+            this.nValuesBeneath = o.nValuesBeneath;
+            this.parent = o.parent;
+            this.childrenCount = o.childrenCount;
+        }
+
+        DictNode(byte[] value, boolean isEndOfValue) {
+            reset(value, isEndOfValue);
+        }
+
+        DictNode(byte[] value, boolean isEndOfValue, ArrayList<DictNode> children) {
+            reset(value, isEndOfValue, children);
+        }
+
+        void reset(byte[] value, boolean isEndOfValue) {
+            reset(value, isEndOfValue, new ArrayList<DictNode>());
+        }
+
+        void reset(byte[] value, boolean isEndOfValue, ArrayList<DictNode> children) {
+            this.part = value;
+            this.isEndOfValue = isEndOfValue;
+            clearChild();
+            for (DictNode child : children) {
+                addChild(child);
+            }
+            this.id = -1;
+        }
+
+        void clearChild() {
+            this.children.clear();
+            int childrenCountDelta = this.childrenCount - 1;
+            for (DictNode p = this; p != null; p = p.parent) {
+                p.childrenCount -= childrenCountDelta;
+            }
+        }
+
+        void addChild(DictNode child) {
+            addChild(-1, child);
+        }
+
+        void addChild(int index, DictNode child) {
+            child.parent = this;
+            if (index < 0) {
+                this.children.add(child);
+            } else {
+                this.children.add(index, child);
+            }
+            for (DictNode p = this; p != null; p = p.parent) {
+                p.childrenCount += child.childrenCount;
+            }
+        }
+
+        public DictNode removeChild(int index) {
+            DictNode child = children.remove(index);
+            child.parent = null;
+            for (DictNode p = this; p != null; p = p.parent) {
+                p.childrenCount -= child.childrenCount;
+            }
+            return child;
+        }
+
+        public DictNode duplicateNode() {
+            DictNode newChild = new DictNode(part, false);
+            newChild.parent = parent;
+            if (parent != null) {
+                int index = parent.children.indexOf(this);
+                parent.addChild(index + 1, newChild);
+            }
+            return newChild;
+        }
+
+        public byte[] firstValue() {
+            ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+            DictNode p = this;
+            while (true) {
+                bytes.write(p.part, 0, p.part.length);
+                if (p.isEndOfValue || p.children.size() == 0) {
+                    break;
+                }
+                p = p.children.get(0);
+            }
+            return bytes.toByteArray();
+        }
+
+        public static DictNode splitNodeTree(DictNode splitNode) {
+            if (splitNode == null) {
+                return null;
+            }
+            DictNode current = splitNode;
+            DictNode p = current.parent;
+            while (p != null) {
+                int index = p.children.indexOf(current);
+                assert index != -1;
+                DictNode newParent = p.duplicateNode();
+                for (int i = p.children.size()-1; i >= index; i--) {
+                    DictNode child = p.removeChild(i);
+                    newParent.addChild(0, child);
+                }
+                current = newParent;
+                p = p.parent;
+            }
+            return current;
+        }
+
+        public static void mergeSingleByteNode(DictNode root, int leftOrRight) {
+            DictNode current = root;
+            DictNode child;
+            while (!current.children.isEmpty()) {
+                child = leftOrRight == 0 ? current.children.get(0) : current.children.get(current.children.size()-1);
+                if (current.children.size() > 1 || current.isEndOfValue) {
+                    current = child;
+                    continue;
+                }
+                byte[] newValue = new byte[current.part.length+child.part.length];
+                System.arraycopy(current.part, 0, newValue, 0, current.part.length);
+                System.arraycopy(child.part, 0, newValue, current.part.length, child.part.length);
+                current.reset(newValue, child.isEndOfValue, child.children);
+                current.id = child.id;
+            }
+        }
+
+        @Override
+        public void write(DataOutput out) throws IOException {
+            byte[] bytes = buildTrieBytes();
+            out.write(bytes);
+        }
+
+        @Override
+        public void readFields(DataInput in) throws IOException {
+            DictNode root = DictSlice.rebuildNodeByDeserialize(in);
+            this.clone(root);
+        }
+
+        protected byte[] buildTrieBytes() {
+            Stats stats = Stats.stats(this);
+            int sizeChildOffset = stats.mbpn_sizeChildOffset;
+            int sizeId = stats.mbpn_sizeId;
+
+            // write head
+            byte[] head;
+            try {
+                ByteArrayOutputStream byteBuf = new ByteArrayOutputStream();
+                DataOutputStream headOut = new DataOutputStream(byteBuf);
+                headOut.write(AppendTrieDictionary.HEAD_MAGIC);
+                headOut.writeShort(0); // head size, will back fill
+                headOut.writeInt(stats.mbpn_footprint); // body size
+                headOut.writeInt(stats.nValues);
+                headOut.write(sizeChildOffset);
+                headOut.write(sizeId);
+                headOut.close();
+                head = byteBuf.toByteArray();
+                BytesUtil.writeUnsigned(head.length, head, AppendTrieDictionary.HEAD_SIZE_I, 2);
+            } catch (IOException e) {
+                throw new RuntimeException(e); // shall not happen, as we are
+            }
+
+            byte[] trieBytes = new byte[stats.mbpn_footprint + head.length];
+            System.arraycopy(head, 0, trieBytes, 0, head.length);
+
+            LinkedList<DictNode> open = new LinkedList<DictNode>();
+            IdentityHashMap<DictNode, Integer> offsetMap = new IdentityHashMap<DictNode, Integer>();
+
+            // write body
+            int o = head.length;
+            offsetMap.put(this, o);
+            o = build_writeNode(this, o, true, sizeChildOffset, sizeId, trieBytes);
+            if (this.children.isEmpty() == false)
+                open.addLast(this);
+
+            while (open.isEmpty() == false) {
+                DictNode parent = open.removeFirst();
+                build_overwriteChildOffset(offsetMap.get(parent), o - head.length, sizeChildOffset, trieBytes);
+                for (int i = 0; i < parent.children.size(); i++) {
+                    DictNode c = parent.children.get(i);
+                    boolean isLastChild = (i == parent.children.size() - 1);
+                    offsetMap.put(c, o);
+                    o = build_writeNode(c, o, isLastChild, sizeChildOffset, sizeId, trieBytes);
+                    if (c.children.isEmpty() == false)
+                        open.addLast(c);
+                }
+            }
+
+            if (o != trieBytes.length)
+                throw new RuntimeException();
+            return trieBytes;
+        }
+
+        private void build_overwriteChildOffset(int parentOffset, int childOffset, int sizeChildOffset, byte[] trieBytes) {
+            int flags = (int) trieBytes[parentOffset] & (TrieDictionary.BIT_IS_LAST_CHILD | TrieDictionary.BIT_IS_END_OF_VALUE);
+            BytesUtil.writeUnsigned(childOffset, trieBytes, parentOffset, sizeChildOffset);
+            trieBytes[parentOffset] |= flags;
+        }
+
+        private int build_writeNode(DictNode n, int offset, boolean isLastChild, int sizeChildOffset, int sizeId, byte[] trieBytes) {
+            int o = offset;
+
+            // childOffset
+            if (isLastChild)
+                trieBytes[o] |= TrieDictionary.BIT_IS_LAST_CHILD;
+            if (n.isEndOfValue)
+                trieBytes[o] |= TrieDictionary.BIT_IS_END_OF_VALUE;
+            o += sizeChildOffset;
+
+            // nValueBytes
+            if (n.part.length > 255)
+                throw new RuntimeException();
+            BytesUtil.writeUnsigned(n.part.length, trieBytes, o, 1);
+            o++;
+
+            // valueBytes
+            System.arraycopy(n.part, 0, trieBytes, o, n.part.length);
+            o += n.part.length;
+
+            if (n.isEndOfValue) {
+                assert n.id > 0;
+                BytesUtil.writeUnsigned(n.id, trieBytes, o, sizeId);
+                o += sizeId;
+            }
+
+            return o;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("DictNode[root=%s, nodes=%d, firstValue=%s]", Bytes.toStringBinary(part), childrenCount, Bytes.toStringBinary(firstValue()));
+        }
+    }
+
+    public static class Stats {
+        public interface Visitor {
+            void visit(DictNode n, int level);
+        }
+
+        private static void traverseR(DictNode node, Visitor visitor, int level) {
+            visitor.visit(node, level);
+            for (DictNode c : node.children)
+                traverseR(c, visitor, level + 1);
+        }
+
+        private static void traversePostOrderR(DictNode node, Visitor visitor, int level) {
+            for (DictNode c : node.children)
+                traversePostOrderR(c, visitor, level + 1);
+            visitor.visit(node, level);
+        }
+
+
+        public int nValues; // number of values in total
+        public int nValueBytesPlain; // number of bytes for all values
+        // uncompressed
+        public int nValueBytesCompressed; // number of values bytes in Trie
+        // (compressed)
+        public int maxValueLength; // size of longest value in bytes
+
+        // the trie is multi-byte-per-node
+        public int mbpn_nNodes; // number of nodes in trie
+        public int mbpn_trieDepth; // depth of trie
+        public int mbpn_maxFanOut; // the maximum no. children
+        public int mbpn_nChildLookups; // number of child lookups during lookup
+        // every value once
+        public int mbpn_nTotalFanOut; // the sum of fan outs during lookup every
+        // value once
+        public int mbpn_sizeValueTotal; // the sum of value space in all nodes
+        public int mbpn_sizeNoValueBytes; // size of field noValueBytes
+        public int mbpn_sizeChildOffset; // size of field childOffset, points to
+        // first child in flattened array
+        public int mbpn_sizeId;          // size of id value, always be 4
+        public int mbpn_footprint; // MBPN footprint in bytes
+
+        /**
+         * out print some statistics of the trie and the dictionary built from it
+         */
+        public static Stats stats(DictNode root) {
+            // calculate nEndValueBeneath
+            traversePostOrderR(root, new Visitor() {
+                @Override
+                public void visit(DictNode n, int level) {
+                    n.nValuesBeneath = n.isEndOfValue ? 1 : 0;
+                    for (DictNode c : n.children)
+                        n.nValuesBeneath += c.nValuesBeneath;
+                }
+            }, 0);
+
+            // run stats
+            final Stats s = new Stats();
+            final ArrayList<Integer> lenAtLvl = new ArrayList<Integer>();
+            traverseR(root, new Visitor() {
+                @Override
+                public void visit(DictNode n, int level) {
+                    if (n.isEndOfValue)
+                        s.nValues++;
+                    s.nValueBytesPlain += n.part.length * n.nValuesBeneath;
+                    s.nValueBytesCompressed += n.part.length;
+                    s.mbpn_nNodes++;
+                    if (s.mbpn_trieDepth < level + 1)
+                        s.mbpn_trieDepth = level + 1;
+                    if (n.children.size() > 0) {
+                        if (s.mbpn_maxFanOut < n.children.size())
+                            s.mbpn_maxFanOut = n.children.size();
+                        int childLookups = n.nValuesBeneath - (n.isEndOfValue ? 1 : 0);
+                        s.mbpn_nChildLookups += childLookups;
+                        s.mbpn_nTotalFanOut += childLookups * n.children.size();
+                    }
+
+                    if (level < lenAtLvl.size())
+                        lenAtLvl.set(level, n.part.length);
+                    else
+                        lenAtLvl.add(n.part.length);
+                    int lenSoFar = 0;
+                    for (int i = 0; i <= level; i++)
+                        lenSoFar += lenAtLvl.get(i);
+                    if (lenSoFar > s.maxValueLength)
+                        s.maxValueLength = lenSoFar;
+                }
+            }, 0);
+
+            // flatten trie footprint calculation, case of Multi-Byte-Per-DictNode
+            s.mbpn_sizeId = 4;
+            s.mbpn_sizeValueTotal = s.nValueBytesCompressed + s.nValues * s.mbpn_sizeId;
+            s.mbpn_sizeNoValueBytes = 1;
+            s.mbpn_sizeChildOffset = 4;
+            s.mbpn_footprint = s.mbpn_sizeValueTotal + s.mbpn_nNodes * (s.mbpn_sizeNoValueBytes + s.mbpn_sizeChildOffset);
+            while (true) { // minimize the offset size to match the footprint
+                int t = s.mbpn_sizeValueTotal + s.mbpn_nNodes * (s.mbpn_sizeNoValueBytes + s.mbpn_sizeChildOffset - 1);
+                // *4 because 2 MSB of offset is used for isEndOfValue & isEndChild flag
+                if (BytesUtil.sizeForValue(t * 4) <= s.mbpn_sizeChildOffset - 1) {
+                    s.mbpn_sizeChildOffset--;
+                    s.mbpn_footprint = t;
+                } else
+                    break;
+            }
+
+            return s;
+        }
+
+        /**
+         * out print trie for debug
+         */
+        public void print(DictNode root) {
+            print(root, System.out);
+        }
+
+        public void print(DictNode root, final PrintStream out) {
+            traverseR(root, new Visitor() {
+                @Override
+                public void visit(DictNode n, int level) {
+                    try {
+                        for (int i = 0; i < level; i++)
+                            out.print("  ");
+                        out.print(new String(n.part, "UTF-8"));
+                        out.print(" - ");
+                        if (n.nValuesBeneath > 0)
+                            out.print(n.nValuesBeneath);
+                        if (n.isEndOfValue)
+                            out.print("* [" + n.id + "]");
+                        out.print("\n");
+                    } catch (UnsupportedEncodingException e) {
+                        e.printStackTrace();
+                    }
+                }
+            }, 0);
+        }
+    }
+
+    public static class Builder<T> {
+        private String baseDir;
+        private int maxId;
+        private int maxValueLength;
+        private int nValues;
+        private BytesConverter<T> bytesConverter;
+
+        private AppendTrieDictionary dict;
+
+        private TreeMap<DictSliceKey, DictNode> dictSliceMap;
+        private static int MAX_ENTRY_IN_SLICE = 10_000_000;
+        private static final double MAX_ENTRY_OVERHEAD_FACTOR = 1.0;
+
+        private int processedCount = 0;
+
+        public static Builder create(String baseDir) throws IOException {
+            return new Builder<>(null, baseDir, 0, 0, 0, new StringBytesConverter(), null);
+        }
+
+        public static Builder create(AppendTrieDictionary dict) throws IOException {
+            return new Builder<>(dict, dict.baseDir, dict.maxId, dict.maxValueLength, dict.nValues, dict.bytesConverter, dict.writeDictMap());
+        }
+
+        // Constructor for a new Dict
+        private Builder(AppendTrieDictionary dict, String baseDir, int maxId, int maxValueLength, int nValues, BytesConverter<T> bytesConverter, byte[] dictMapBytes) throws IOException {
+            this.dict = dict;
+            this.baseDir = baseDir;
+            this.maxId = maxId;
+            this.maxValueLength = maxValueLength;
+            this.nValues = nValues;
+            this.bytesConverter = bytesConverter;
+
+            MAX_ENTRY_IN_SLICE = KylinConfig.getInstanceFromEnv().getAppendDictEntrySize();
+            int cacheSize = KylinConfig.getInstanceFromEnv().getAppendDictCacheSize();
+            // create a new cached map with baseDir
+            dictSliceMap = CachedTreeMap.CachedTreeMapBuilder.newBuilder().maxSize(cacheSize)
+                    .baseDir(baseDir).keyClazz(DictSliceKey.class).valueClazz(DictNode.class)
+                    .persistent(true).immutable(false).build();
+            if (dictMapBytes != null) {
+                ((Writable)dictSliceMap).readFields(new DataInputStream(new ByteArrayInputStream(dictMapBytes)));
+            }
+        }
+
+        public void addValue(T value) {
+            addValue(bytesConverter.convertToBytes(value));
+        }
+
+        public void addValue(byte[] value) {
+            if (++processedCount % 1_000_000 == 0) {
+                logger.debug("add value count " + processedCount);
+            }
+            maxValueLength = Math.max(maxValueLength, value.length);
+
+            if (dictSliceMap.isEmpty()) {
+                DictNode root = new DictNode(new byte[0], false);
+                dictSliceMap.put(DictSliceKey.wrap(new byte[0]), root);
+            }
+            DictSliceKey sliceKey = dictSliceMap.floorKey(DictSliceKey.wrap(value));
+            if (sliceKey == null) {
+                sliceKey = dictSliceMap.firstKey();
+            }
+            DictNode root = dictSliceMap.get(sliceKey);
+            addValueR(root, value, 0);
+            if (root.childrenCount > MAX_ENTRY_IN_SLICE * MAX_ENTRY_OVERHEAD_FACTOR) {
+                dictSliceMap.remove(sliceKey);
+                DictNode newRoot = splitNodeTree(root);
+                DictNode.mergeSingleByteNode(root, 1);
+                DictNode.mergeSingleByteNode(newRoot, 0);
+                dictSliceMap.put(DictSliceKey.wrap(root.firstValue()), root);
+                dictSliceMap.put(DictSliceKey.wrap(newRoot.firstValue()), newRoot);
+            }
+        }
+
+        private DictNode splitNodeTree(DictNode root) {
+            DictNode parent = root;
+            DictNode splitNode;
+            int childCountToSplit = (int)(MAX_ENTRY_IN_SLICE * MAX_ENTRY_OVERHEAD_FACTOR / 2);
+            while (true) {
+                List<DictNode> children = parent.children;
+                if (children.size() == 0){
+                    splitNode = parent;
+                    break;
+                } else if (children.size() == 1) {
+                    parent = children.get(0);
+                    continue;
+                } else {
+                    for (int i = children.size()-1; i >= 0; i--) {
+                        parent = children.get(i);
+                        if (childCountToSplit > children.get(i).childrenCount) {
+                            childCountToSplit -= children.get(i).childrenCount;
+                        } else {
+                            childCountToSplit --;
+                            break;
+                        }
+                    }
+                }
+            }
+            return DictNode.splitNodeTree(splitNode);
+        }
+
+        private int createNextId() {
+            int id = ++maxId;
+            if (maxId < 0) {
+                throw new IllegalArgumentException("AppendTrieDictionary Id overflow Integer.MAX_VALUE");
+            }
+            nValues ++;
+            return id;
+        }
+
+        private void addValueR(DictNode node, byte[] value, int start) {
+            assert value.length - start <= 255 : "value bytes overflow than 255";
+            // match the value part of current node
+            int i = 0, j = start;
+            int n = node.part.length, nn = value.length;
+            int comp = 0;
+            for (; i < n && j < nn; i++, j++) {
+                comp = BytesUtil.compareByteUnsigned(node.part[i], value[j]);
+                if (comp != 0)
+                    break;
+            }
+
+            // if value fully matched within the current node
+            if (j == nn) {
+                // if equals to current node, just mark end of value
+                if (i == n) {
+                    // if the first match, assign an Id to nodt
+                    if (!node.isEndOfValue) {
+                        node.id = createNextId();
+                    }
+                    node.isEndOfValue = true;
+                }
+                // otherwise, split the current node into two
+                else {
+                    DictNode c = new DictNode(BytesUtil.subarray(node.part, i, n), node.isEndOfValue, node.children);
+                    c.id = node.id;
+                    node.reset(BytesUtil.subarray(node.part, 0, i), true);
+                    node.addChild(c);
+                    node.id = createNextId();
+                }
+                return;
+            }
+
+            // if partially matched the current, split the current node, add the new
+            // value, make a 3-way
+            if (i < n) {
+                DictNode c1 = new DictNode(BytesUtil.subarray(node.part, i, n), node.isEndOfValue, node.children);
+                c1.id = node.id;
+                DictNode c2 = new DictNode(BytesUtil.subarray(value, j, nn), true);
+                c2.id = createNextId();
+                node.reset(BytesUtil.subarray(node.part, 0, i), false);
+                if (comp < 0) {
+                    node.addChild(c1);
+                    node.addChild(c2);
+                } else {
+                    node.addChild(c2);
+                    node.addChild(c1);
+                }
+                return;
+            }
+
+            // out matched the current, binary search the next byte for a child node
+            // to continue
+            byte lookfor = value[j];
+            int lo = 0;
+            int hi = node.children.size() - 1;
+            int mid = 0;
+            boolean found = false;
+            comp = 0;
+            while (!found && lo <= hi) {
+                mid = lo + (hi - lo) / 2;
+                DictNode c = node.children.get(mid);
+                comp = BytesUtil.compareByteUnsigned(lookfor, c.part[0]);
+                if (comp < 0)
+                    hi = mid - 1;
+                else if (comp > 0)
+                    lo = mid + 1;
+                else
+                    found = true;
+            }
+            // found a child node matching the first byte, continue in that child
+            if (found) {
+                addValueR(node.children.get(mid), value, j);
+            }
+            // otherwise, make the value a new child
+            else {
+                DictNode c = new DictNode(BytesUtil.subarray(value, j, nn), true);
+                c.id = createNextId();
+                node.addChild(comp <= 0 ? mid : mid + 1, c);
+            }
+        }
+
+        public AppendTrieDictionary<T> build(int baseId) throws IOException {
+            ByteArrayOutputStream buf = new ByteArrayOutputStream();
+            DataOutputStream out = new DataOutputStream(buf);
+            ((Writable)dictSliceMap).write(out);
+            byte[] dictMapBytes = buf.toByteArray();
+            buf.close();
+            out.close();
+
+            if (dict == null) {
+                dict = new AppendTrieDictionary<T>();
+            }
+            dict.update(baseDir, baseId, maxId, maxValueLength, nValues, bytesConverter, dictMapBytes);
+            dict.flushIndex();
+
+            return dict;
+        }
+    }
+
+
+    @Override
+    protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
+        if (dictSliceMap.isEmpty()) {
+            return -1;
+        }
+        byte[] tempVal = new byte[len];
+        System.arraycopy(value, offset, tempVal, 0, len);
+        DictSliceKey sliceKey = dictSliceMap.floorKey(DictSliceKey.wrap(tempVal));
+        if (sliceKey == null) {
+            sliceKey = dictSliceMap.firstKey();
+        }
+        DictSlice slice = dictSliceMap.get(sliceKey);
+        int id = slice.getIdFromValueBytesImpl(value, offset, len, roundingFlag);
+        if (id < 0)
+            logger.error("Not a valid value: " + bytesConverter.convertFromBytes(value, offset, len));
+        return id;
+    }
+
+    @Override
+    public int getMinId() {
+        return baseId;
+    }
+
+    @Override
+    public int getMaxId() {
+        return maxId;
+    }
+
+    @Override
+    public int getSizeOfId() {
+        return 4;
+    }
+
+    @Override
+    public int getSizeOfValue() {
+        return maxValueLength;
+    }
+
+    @Override
+    final protected int getIdFromValueImpl(T value, int roundingFlag) {
+        if (enableValueCache && roundingFlag == 0) {
+            HashMap cache = valueToIdCache.get(); // SoftReference to skip cache gracefully when short of memory
+            if (cache != null) {
+                Integer id = null;
+                id = (Integer) cache.get(value);
+                if (id != null)
+                    return id.intValue();
+
+                byte[] valueBytes = bytesConverter.convertToBytes(value);
+                id = getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
+
+                cache.put(value, id);
+                return id;
+            }
+        }
+        byte[] valueBytes = bytesConverter.convertToBytes(value);
+        return getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
+    }
+
+    @Override
+    final protected T getValueFromIdImpl(int id) {
+        throw new UnsupportedOperationException("AppendTrieDictionary can't retrive value from id");
+    }
+
+    @Override
+    protected byte[] getValueBytesFromIdImpl(int id) {
+        throw new UnsupportedOperationException("AppendTrieDictionary can't retrive value from id");
+    }
+
+    @Override
+    protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) {
+        throw new UnsupportedOperationException("AppendTrieDictionary can't retrive value from id");
+    }
+
+    public void flushIndex() throws IOException {
+        Path filePath = new Path(baseDir+"/.index");
+        Configuration conf = new Configuration();
+        try (FSDataOutputStream indexOut = (FileSystem.get(filePath.toUri(), conf)).create(filePath, true, 8*1024*1024, (short)2, 8*1024*1024*8)) {
+            indexOut.writeInt(baseId);
+            indexOut.writeInt(maxId);
+            indexOut.writeInt(maxValueLength);
+            indexOut.writeInt(nValues);
+            indexOut.writeUTF(bytesConverter.getClass().getName());
+            ((Writable)dictSliceMap).write(indexOut);
+        }
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        out.writeUTF(baseDir);
+        flushIndex();
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+        String baseDir = in.readUTF();
+        Path filePath = new Path(baseDir+"/.index");
+        Configuration conf = new Configuration();
+        try (FSDataInputStream input = (FileSystem.get(filePath.toUri(), conf)).open(filePath, 8*1024*1024)) {
+            int baseId = input.readInt();
+            int maxId = input.readInt();
+            int maxValueLength = input.readInt();
+            int nValues = input.readInt();
+            String converterName = input.readUTF();
+            BytesConverter converter = null;
+            if (converterName.isEmpty() == false) {
+                try {
+                    converter = ClassUtil.forName(converterName, BytesConverter.class).newInstance();
+                } catch (Exception e) {
+                    throw new IOException(e);
+                }
+            }
+            update(baseDir, baseId, maxId, maxValueLength, nValues, converter, input);
+        }
+    }
+
+    @Override
+    public void dump(PrintStream out) {
+        out.println("Total " + nValues + " values, " + (dictSliceMap == null ? 0 : dictSliceMap.size()) + " slice");
+    }
+
+    @Override
+    public int hashCode() {
+        int hashCode = 31;
+        for (DictSlice slice : dictSliceMap.values()) {
+            hashCode += 31 * slice.hashCode();
+        }
+        return hashCode;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public boolean contains(Dictionary other) {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java b/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java
new file mode 100644
index 0000000..81ac82f
--- /dev/null
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/CachedTreeMap.java
@@ -0,0 +1,348 @@
+/*
+ * 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.dict;
+
+import com.google.common.cache.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.persistence.ComparableWritable;
+import org.apache.kylin.common.persistence.Writable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Created by sunyerui on 16/5/2.
+ * TODO Depends on HDFS for now, ideally just depends on storage interface
+ */
+public class CachedTreeMap<K extends ComparableWritable, V extends Writable> extends TreeMap<K, V> implements Writable {
+    private static final Logger logger = LoggerFactory.getLogger(CachedTreeMap.class);
+
+    private final Class<K> keyClazz;
+    private final Class<V> valueClazz;
+    transient volatile Collection<V> values;
+    private final LoadingCache<K, V> valueCache;
+    private final TreeSet<String> fileList;
+    private final Configuration conf;
+    private final String baseDir;
+    private final boolean persistent;
+    private final boolean immutable;
+    private long writeValueTime = 0;
+    private long readValueTime = 0;
+
+    private static final int BUFFER_SIZE = 8 * 1024 * 1024;
+
+    public static class CachedTreeMapBuilder<K, V> {
+        private Class<K> keyClazz;
+        private Class<V> valueClazz;
+        private int maxCount = 8;
+        private String baseDir;
+        private boolean persistent;
+        private boolean immutable;
+
+        public static CachedTreeMapBuilder newBuilder() {
+            return new CachedTreeMapBuilder();
+        }
+
+        private CachedTreeMapBuilder() {}
+
+        public CachedTreeMapBuilder keyClazz(Class<K> clazz) {
+            this.keyClazz = clazz;
+            return this;
+        }
+
+        public CachedTreeMapBuilder valueClazz(Class<V> clazz) {
+            this.valueClazz = clazz;
+            return this;
+        }
+
+        public CachedTreeMapBuilder<K, V> maxSize(int maxCount) {
+            this.maxCount = maxCount;
+            return this;
+        }
+
+        public CachedTreeMapBuilder<K, V> baseDir(String baseDir) {
+            this.baseDir = baseDir;
+            return this;
+        }
+
+        public CachedTreeMapBuilder<K, V> persistent(boolean persistent) {
+            this.persistent = persistent;
+            return this;
+        }
+
+        public CachedTreeMapBuilder<K, V> immutable(boolean immutable) {
+            this.immutable = immutable;
+            return this;
+        }
+
+        public CachedTreeMap build() {
+            if (baseDir == null) {
+                throw new RuntimeException("CachedTreeMap need a baseDir to cache data");
+            }
+            if (keyClazz == null || valueClazz == null) {
+                throw new RuntimeException("CachedTreeMap need key and value clazz to serialize data");
+            }
+            CachedTreeMap map = new CachedTreeMap(maxCount, keyClazz, valueClazz, baseDir, persistent, immutable);
+            return map;
+        }
+    }
+
+    private CachedTreeMap(int maxCount, Class<K> keyClazz, Class<V> valueClazz, String baseDir, boolean persistent, boolean immutable) {
+        super();
+        this.keyClazz = keyClazz;
+        this.valueClazz = valueClazz;
+        this.fileList = new TreeSet<>();
+        this.conf = new Configuration();
+        this.baseDir = baseDir;
+        this.persistent = persistent;
+        this.immutable = immutable;
+        CacheBuilder builder = CacheBuilder.newBuilder().removalListener(new RemovalListener<K, V>() {
+            @Override
+            public void onRemoval(RemovalNotification<K, V> notification) {
+                logger.info(String.format("Evict cache key %s(%d) with value %s caused by %s, size %d/%d ",
+                        notification.getKey(), notification.getKey().hashCode(), notification.getValue(), notification.getCause(),
+                        size(), valueCache.size()));
+                switch (notification.getCause()) {
+                    case SIZE:
+                        writeValue(notification.getKey(), notification.getValue());
+                        break;
+                    case EXPLICIT:
+                        // skip delete files to recover from error during dict appending
+                        // deleteValue(notification.getKey());
+                        break;
+                    default:
+                        throw new RuntimeException("unexpected evict reason " + notification.getCause());
+                }
+            }
+        }).maximumSize(maxCount);
+        // For immutable values, use soft reference to free memory when gc, and just load again when need it
+        if (this.immutable) {
+            builder.softValues();
+        }
+        this.valueCache = builder.build(new CacheLoader<K, V>() {
+            @Override
+            public V load(K key) throws Exception {
+                V value = readValue(key);
+                logger.info(String.format("Load cache by key %s(%d) with value %s", key, key.hashCode(), value));
+                return value;
+            }
+        });
+    }
+
+    private String generateFileName(K key) {
+        String file = baseDir + "/cached_" + key.toString();
+        return file;
+    }
+
+    private void writeValue(K key, V value) {
+        if (immutable) {
+            return;
+        }
+        long t0 = System.currentTimeMillis();
+        String fileName = generateFileName(key);
+        Path filePath = new Path(fileName);
+        try (FSDataOutputStream out = (FileSystem.get(filePath.toUri(), conf)).create(filePath, true, BUFFER_SIZE, (short)2, BUFFER_SIZE*8)) {
+            value.write(out);
+            if (!persistent) {
+                FileSystem.get(filePath.toUri(), conf).deleteOnExit(filePath);
+            }
+        } catch (Exception e) {
+            logger.error(String.format("write value into %s exception: %s", fileName, e), e);
+            throw new RuntimeException(e.getCause());
+        } finally {
+            fileList.add(fileName);
+            writeValueTime += System.currentTimeMillis() - t0;
+        }
+    }
+
+    private V readValue(K key) throws Exception {
+        long t0 = System.currentTimeMillis();
+        String fileName = generateFileName(key);
+        Path filePath = new Path(fileName);
+        try (FSDataInputStream input = (FileSystem.get(filePath.toUri(), conf)).open(filePath, BUFFER_SIZE)) {
+            V value = valueClazz.newInstance();
+            value.readFields(input);
+            return value;
+        } catch (Exception e) {
+            logger.error(String.format("read value from %s exception: %s", fileName, e), e);
+            return null;
+        } finally {
+            readValueTime += System.currentTimeMillis() - t0;
+        }
+    }
+
+    private void deleteValue(K key) {
+        if (persistent && immutable) {
+            return;
+        }
+        String fileName = generateFileName(key);
+        Path filePath = new Path(fileName);
+        try {
+            FileSystem fs = FileSystem.get(filePath.toUri(), conf);
+            if (fs.exists(filePath)) {
+                fs.delete(filePath, true);
+            }
+        } catch (Exception e) {
+            logger.error(String.format("delete value file %s exception: %s", fileName, e), e);
+        } finally {
+            fileList.remove(fileName);
+        }
+    }
+
+    @Override
+    public V put(K key, V value) {
+        super.put(key, null);
+        valueCache.put(key, value);
+        return null;
+    }
+
+    @Override
+    public V get(Object key) {
+        if (super.containsKey(key)) {
+            try {
+                return valueCache.get((K)key);
+            } catch (ExecutionException e) {
+                logger.error(String.format("get value with key %s exception: ", key, e), e);
+                return null;
+            }
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public V remove(Object key) {
+        super.remove(key);
+        valueCache.invalidate(key);
+        return null;
+    }
+
+    @Override
+    public void clear() {
+        super.clear();
+        values = null;
+        valueCache.invalidateAll();
+    }
+
+    public Collection<V> values() {
+        Collection<V> vs = values;
+        return (vs != null) ? vs : (values = new Values());
+    }
+
+    class Values extends AbstractCollection<V> {
+        @Override
+        public Iterator<V> iterator() {
+            return new ValueIterator<>();
+        }
+
+        @Override
+        public int size() {
+            return CachedTreeMap.this.size();
+        }
+    }
+
+    class ValueIterator<V> implements Iterator<V> {
+        Iterator<K> keyIterator;
+        K currentKey;
+
+        public ValueIterator() {
+            keyIterator = CachedTreeMap.this.keySet().iterator();
+        }
+
+        @Override
+        public boolean hasNext() {
+            return keyIterator.hasNext();
+        }
+
+        @Override
+        public V next() {
+            currentKey = keyIterator.next();
+            try {
+                return (V)valueCache.get(currentKey);
+            } catch (ExecutionException e) {
+                logger.error(String.format("get value with key %s exception: ", currentKey, e), e);
+                return null;
+            }
+        }
+
+        @Override
+        public void remove() {
+            keyIterator.remove();
+            valueCache.invalidate(currentKey);
+        }
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        assert persistent : "Only support serialize with persistent true";
+        out.writeInt(size());
+        for (K key : keySet()) {
+            key.write(out);
+            V value = valueCache.getIfPresent(key);
+            if (null != value) {
+                writeValue(key, value);
+            }
+        }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+        assert persistent : "Only support deserialize with persistent true";
+        int size = in.readInt();
+        try {
+            for (int i = 0; i < size; i++) {
+                K key = keyClazz.newInstance();
+                key.readFields(in);
+                super.put(key, null);
+            }
+        } catch (Exception e) {
+            throw new IOException(e);
+        }
+    }
+
+    // clean up all tmp files
+    @Override
+    public void finalize() throws Throwable {
+        if (persistent) {
+            return;
+        }
+        try {
+            this.clear();
+            for (String file : fileList) {
+                try {
+                    Path filePath = new Path(file);
+                    FileSystem fs = FileSystem.get(filePath.toUri(), conf);
+                    fs.delete(filePath, true);
+                } catch (Throwable t) {}
+            }
+        } catch (Throwable t) {
+        } finally {
+          super.finalize();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
index 0d1fe88..95b6087 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
@@ -44,7 +44,7 @@ public class DictionaryGenerator {
 
     private static final Logger logger = LoggerFactory.getLogger(DictionaryGenerator.class);
 
-    private static final String[] DATE_PATTERNS = new String[] { "yyyy-MM-dd", "yyyyMMdd" };
+    private static final String[] DATE_PATTERNS = new String[] { "yyyy-MM-dd", "yyyyMMdd" };
 
     private static int getDictionaryMaxCardinality() {
         try {
@@ -70,15 +70,15 @@ public class DictionaryGenerator {
             builder = new StringDictBuilder();
         }
         
-        return buildDictionary(builder, valueEnumerator);
+        return buildDictionary(builder, null, valueEnumerator);
     }
     
-    public static Dictionary<String> buildDictionary(IDictionaryBuilder builder, IDictionaryValueEnumerator valueEnumerator) throws IOException {
+    public static Dictionary<String> buildDictionary(IDictionaryBuilder builder, DictionaryInfo dictInfo, IDictionaryValueEnumerator valueEnumerator) throws IOException {
         int baseId = 0; // always 0 for now
         int nSamples = 5;
         ArrayList<String> samples = new ArrayList<String>(nSamples);
 
-        Dictionary<String> dict = builder.build(valueEnumerator, baseId, nSamples, samples);
+        Dictionary<String> dict = builder.build(dictInfo, valueEnumerator, baseId, nSamples, samples);
 
         // log a few samples
         StringBuilder buf = new StringBuilder();
@@ -89,20 +89,22 @@ public class DictionaryGenerator {
             buf.append(s.toString()).append("=>").append(dict.getIdFromValue(s));
         }
         logger.debug("Dictionary value samples: " + buf.toString());
-        logger.debug("Dictionary cardinality " + dict.getSize());
+        logger.debug("Dictionary cardinality: " + dict.getSize());
+        logger.debug("Dictionary builder class: " + builder.getClass().getName());
+        logger.debug("Dictionary class: " + dict.getClass().getName());
         if (dict instanceof TrieDictionary && dict.getSize() > DICT_MAX_CARDINALITY) {
             throw new IllegalArgumentException("Too high cardinality is not suitable for dictionary -- cardinality: " + dict.getSize());
         }
         return dict;
     }
 
-    public static Dictionary mergeDictionaries(DataType dataType, List<DictionaryInfo> sourceDicts) throws IOException {
-        return buildDictionary(dataType, new MultipleDictionaryValueEnumerator(sourceDicts));
+    public static Dictionary mergeDictionaries(DataType dataType, List<DictionaryInfo> sourceDicts) throws IOException {
+        return buildDictionary(dataType, new MultipleDictionaryValueEnumerator(sourceDicts));
     }
 
     private static class DateDictBuilder implements IDictionaryBuilder {
         @Override
-        public Dictionary<String> build(IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList<String> returnSamples) throws IOException {
+        public Dictionary<String> build(DictionaryInfo dictInfo, IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList<String> returnSamples) throws IOException {
             final int BAD_THRESHOLD = 0;
             String matchPattern = null;
             byte[] value;
@@ -141,14 +143,14 @@ public class DictionaryGenerator {
     
     private static class TimeDictBuilder implements IDictionaryBuilder {
         @Override
-        public Dictionary<String> build(IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList<String> returnSamples) throws IOException {
+        public Dictionary<String> build(DictionaryInfo dictInfo, IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList<String> returnSamples) throws IOException {
             return new TimeStrDictionary(); // base ID is always 0
         }
     }
 
     private static class StringDictBuilder implements IDictionaryBuilder {
         @Override
-        public Dictionary<String> build(IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList<String> returnSamples) throws IOException {
+        public Dictionary<String> build(DictionaryInfo dictInfo, IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList<String> returnSamples) throws IOException {
             TrieDictionaryBuilder builder = new TrieDictionaryBuilder(new StringBytesConverter());
             byte[] value;
             while (valueEnumerator.moveNext()) {
@@ -166,7 +168,7 @@ public class DictionaryGenerator {
 
     private static class NumberDictBuilder implements IDictionaryBuilder {
         @Override
-        public Dictionary<String> build(IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList<String> returnSamples) throws IOException {
+        public Dictionary<String> build(DictionaryInfo dictInfo, IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList<String> returnSamples) throws IOException {
             NumberDictionaryBuilder builder = new NumberDictionaryBuilder(new StringBytesConverter());
             byte[] value;
             while (valueEnumerator.moveNext()) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
index 43f7b26..5668bd8 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
@@ -312,7 +312,7 @@ public class DictionaryManager {
             if (builderClass == null)
                 dictionary = DictionaryGenerator.buildDictionary(DataType.getType(dictInfo.getDataType()), columnValueEnumerator);
             else
-                dictionary = DictionaryGenerator.buildDictionary((IDictionaryBuilder) ClassUtil.newInstance(builderClass), columnValueEnumerator);
+                dictionary = DictionaryGenerator.buildDictionary((IDictionaryBuilder) ClassUtil.newInstance(builderClass), dictInfo, columnValueEnumerator);
         } finally {
             if (columnValueEnumerator != null)
                 columnValueEnumerator.close();

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-dictionary/src/main/java/org/apache/kylin/dict/GlobalDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/GlobalDictionaryBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/GlobalDictionaryBuilder.java
new file mode 100644
index 0000000..0f4d8bb
--- /dev/null
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/GlobalDictionaryBuilder.java
@@ -0,0 +1,85 @@
+/*
+ * 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.dict;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.metadata.MetadataManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.NavigableSet;
+
+/**
+ * GlobalDictinary based on whole cube, to ensure one value has same dict id in different segments.
+ * GlobalDictinary mainly used for count distinct measure to support rollup among segments.
+ * Created by sunyerui on 16/5/24.
+ */
+public class GlobalDictionaryBuilder implements IDictionaryBuilder {
+    private static final Logger logger = LoggerFactory.getLogger(GlobalDictionaryBuilder.class);
+
+    @Override
+    public Dictionary<String> build(DictionaryInfo dictInfo, IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList<String> returnSamples) throws IOException {
+        if (dictInfo == null) {
+            throw new IllegalArgumentException("GlobalDictinaryBuilder must used with an existing DictionaryInfo");
+        }
+        String dictDir = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "/resources/GlobalDict/" + dictInfo.getResourceDir() + "/";
+
+        // Try to load the existing dict from cache, making sure there's only the same one object in memory
+        NavigableSet<String> dicts = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getStore().listResources(dictInfo.getResourceDir());
+        ArrayList<String> appendDicts = new ArrayList<>();
+        if (dicts != null && !dicts.isEmpty()) {
+            for (String dict : dicts) {
+                DictionaryInfo info = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getStore().getResource(dict, DictionaryInfo.class, DictionaryInfoSerializer.INFO_SERIALIZER);
+                if (info.getDictionaryClass().equals(AppendTrieDictionary.class.getName())) {
+                    appendDicts.add(dict);
+                }
+            }
+        }
+
+        AppendTrieDictionary.Builder<String> builder;
+        if (appendDicts.isEmpty()) {
+            logger.info("GlobalDict {} is empty, create new one", dictInfo.getResourceDir());
+            builder = AppendTrieDictionary.Builder.create(dictDir);
+        } else if (appendDicts.size() == 1) {
+            logger.info("GlobalDict {} exist, append value", appendDicts.get(0));
+            AppendTrieDictionary dict = (AppendTrieDictionary)DictionaryManager.getInstance(KylinConfig.getInstanceFromEnv()).getDictionary(appendDicts.get(0));
+            builder = AppendTrieDictionary.Builder.create(dict);
+        } else {
+            throw new IllegalStateException(String.format("GlobalDict %s should have 0 or 1 append dict but %d", dictInfo.getResourceDir(), appendDicts.size()));
+        }
+
+        byte[] value;
+        while (valueEnumerator.moveNext()) {
+            value = valueEnumerator.current();
+            if (value == null) {
+                continue;
+            }
+            String v = Bytes.toString(value);
+            builder.addValue(v);
+            if (returnSamples.size() < nSamples && returnSamples.contains(v) == false)
+                returnSamples.add(v);
+        }
+        return builder.build(baseId);
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-dictionary/src/main/java/org/apache/kylin/dict/IDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/IDictionaryBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/IDictionaryBuilder.java
index 9ace239..8f95a2a 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/IDictionaryBuilder.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/IDictionaryBuilder.java
@@ -25,5 +25,5 @@ import org.apache.kylin.common.util.Dictionary;
 
 public interface IDictionaryBuilder {
 
-    Dictionary<String> build(IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList<String> returnSamples) throws IOException;
+    Dictionary<String> build(DictionaryInfo dictInfo, IDictionaryValueEnumerator valueEnumerator, int baseId, int nSamples, ArrayList<String> returnSamples) throws IOException;
 }


[2/6] kylin git commit: KYLIN-1705 Global (and more scalable) dictionary

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
new file mode 100644
index 0000000..10bbb77
--- /dev/null
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
@@ -0,0 +1,229 @@
+package org.apache.kylin.dict;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.KylinConfig;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.io.*;
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Created by sunyerui on 16/4/28.
+ */
+public class AppendTrieDictionaryTest {
+
+    @BeforeClass
+    public static void setUp() {
+        KylinConfig.destroyInstance();
+        System.setProperty(KylinConfig.KYLIN_CONF, "../examples/test_case_data/localmeta");
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        config.setAppendDictEntrySize(50000);
+        config.setAppendDictCacheSize(3);
+        config.setProperty("kylin.hdfs.working.dir", "/tmp");
+    }
+
+    @AfterClass
+    public static void tearDown() {
+        String workingDir = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory();
+        try {
+            FileSystem.get(new Path(workingDir).toUri(), new Configuration()).delete(new Path(workingDir), true);
+        } catch (IOException e) {}
+    }
+
+    public static final String[] words = new String[]{
+        "paint", "par", "part", "parts", "partition", "partitions",
+        "party", "partie", "parties", "patient",
+        "taste", "tar", "trie", "try", "tries",
+        "\u5b57\u5178", "\u5b57\u5178\u6811", "\u5b57\u6bcd", // non-ascii characters
+        "",  // empty
+        "paint", "tar", "try", // some dup
+    };
+
+    @Test
+    public void testStringRepeatly() throws IOException {
+        ArrayList<String> list = new ArrayList<>();
+        Collections.addAll(list, words);
+        ArrayList<String> notfound = new ArrayList<>();
+        notfound.add("pa");
+        notfound.add("pars");
+        notfound.add("tri");
+        notfound.add("\u5b57");
+        for (int i = 0; i < 100; i++) {
+            testStringDictAppend(list, notfound, true);
+        }
+    }
+
+    @Test
+    public void englishWordsTest() throws Exception {
+        InputStream is = new FileInputStream("src/test/resources/dict/english-words.80 (scowl-2015.05.18).txt");
+        ArrayList<String> str = loadStrings(is);
+        testStringDictAppend(str, null, false);
+    }
+
+    @Test
+    public void categoryNamesTest() throws Exception {
+        InputStream is = new FileInputStream("src/test/resources/dict/dw_category_grouping_names.dat");
+        ArrayList<String> str = loadStrings(is);
+        testStringDictAppend(str, null, true);
+    }
+
+    private static ArrayList<String> loadStrings(InputStream is) throws Exception {
+        ArrayList<String> r = new ArrayList<String>();
+        BufferedReader reader = new BufferedReader(new InputStreamReader(is, "UTF-8"));
+        try {
+            String word;
+            while ((word = reader.readLine()) != null) {
+                word = word.trim();
+                if (word.isEmpty() == false)
+                    r.add(word);
+            }
+        } finally {
+            reader.close();
+            is.close();
+        }
+        return r;
+    }
+
+    @Ignore("need huge key set")
+    @Test
+    public void testHugeKeySet() throws IOException {
+        BytesConverter converter = new StringBytesConverter();
+        AppendTrieDictionary.Builder<String> b = AppendTrieDictionary.Builder.create("/tmp");
+        AppendTrieDictionary<String> dict = null;
+
+        InputStream is = new FileInputStream("src/test/resources/dict/huge_key");
+        BufferedReader reader = new BufferedReader(new InputStreamReader(is, "UTF-8"));
+        try {
+            String word;
+            while ((word = reader.readLine()) != null) {
+                word = word.trim();
+                if (!word.isEmpty());
+                    b.addValue(word);
+            }
+        } finally {
+            reader.close();
+            is.close();
+        }
+        dict = b.build(0);
+        dict.dump(System.out);
+    }
+
+    private static void testStringDictAppend(ArrayList<String> list, ArrayList<String> notfound, boolean shuffleList) throws IOException {
+        Random rnd = new Random(System.currentTimeMillis());
+        ArrayList<String> strList = new ArrayList<String>();
+        strList.addAll(list);
+        if (shuffleList) {
+            Collections.shuffle(strList, rnd);
+        }
+        BytesConverter converter = new StringBytesConverter();
+
+        AppendTrieDictionary.Builder<String> b = AppendTrieDictionary.Builder.create("/tmp");
+        AppendTrieDictionary<String> dict = null;
+        TreeMap<Integer, String> checkMap = new TreeMap<>();
+        int firstAppend = rnd.nextInt(strList.size()/2);
+        int secondAppend = firstAppend + rnd.nextInt((strList.size()-firstAppend)/2);
+        int appendIndex = 0;
+        int checkIndex = 0;
+
+        for (; appendIndex < firstAppend; appendIndex++) {
+            b.addValue(strList.get(appendIndex));
+        }
+        dict = b.build(0);
+        dict.dump(System.out);
+        for (;checkIndex < firstAppend; checkIndex++) {
+            String str = strList.get(checkIndex);
+            byte[] bytes = converter.convertToBytes(str);
+            int id = dict.getIdFromValueBytesImpl(bytes, 0, bytes.length, 0);
+            assertFalse(String.format("Id %d for %s should be empty, but is %s", id, str, checkMap.get(id)),
+                    checkMap.containsKey(id) && !str.equals(checkMap.get(id)));
+            checkMap.put(id, str);
+        }
+
+        // reopen dict and append
+        b = AppendTrieDictionary.Builder.create(dict);
+        for (; appendIndex < secondAppend; appendIndex++) {
+            b.addValue(strList.get(appendIndex));
+        }
+        AppendTrieDictionary newDict = b.build(0);
+        assert newDict == dict;
+        dict = newDict;
+        dict.dump(System.out);
+        checkIndex = 0;
+        for (;checkIndex < secondAppend; checkIndex++) {
+            String str = strList.get(checkIndex);
+            byte[] bytes = converter.convertToBytes(str);
+            int id = dict.getIdFromValueBytesImpl(bytes, 0, bytes.length, 0);
+            if (checkIndex < firstAppend) {
+                assertEquals("Except id " + id + " for " + str + " but " + checkMap.get(id), str, checkMap.get(id));
+            } else {
+                // check second append str, should be new id
+                assertFalse(String.format("Id %d for %s should be empty, but is %s", id, str, checkMap.get(id)),
+                    checkMap.containsKey(id) && !str.equals(checkMap.get(id)));
+                checkMap.put(id, str);
+            }
+        }
+
+        // reopen dict and append rest str
+        b = AppendTrieDictionary.Builder.create(dict);
+        for (; appendIndex < strList.size(); appendIndex++) {
+            b.addValue(strList.get(appendIndex));
+        }
+        newDict = b.build(0);
+        assert newDict == dict;
+        dict = newDict;
+        dict.dump(System.out);
+        checkIndex = 0;
+        for (; checkIndex < strList.size(); checkIndex++) {
+            String str = strList.get(checkIndex);
+            byte[] bytes = converter.convertToBytes(str);
+            int id = dict.getIdFromValueBytesImpl(bytes, 0, bytes.length, 0);
+            if (checkIndex < secondAppend) {
+                assertEquals("Except id " + id + " for " + str + " but " + checkMap.get(id), str, checkMap.get(id));
+            } else {
+                // check third append str, should be new id
+                assertFalse(String.format("Id %d for %s should be empty, but is %s", id, str, checkMap.get(id)),
+                    checkMap.containsKey(id) && !str.equals(checkMap.get(id)));
+                checkMap.put(id, str);
+            }
+        }
+        if (notfound != null) {
+            for (String s : notfound) {
+                byte[] bytes = converter.convertToBytes(s);
+                int id = dict.getIdFromValueBytesImpl(bytes, 0, bytes.length, 0);
+                assertEquals(-1, id);
+            }
+        }
+
+        dict = testSerialize(dict, converter);
+        for (String str : strList) {
+            byte[] bytes = converter.convertToBytes(str);
+            int id = dict.getIdFromValueBytesImpl(bytes, 0, bytes.length, 0);
+            assertEquals("Except id " + id + " for " + str + " but " + checkMap.get(id), str, checkMap.get(id));
+        }
+    }
+
+    private static AppendTrieDictionary<String> testSerialize(AppendTrieDictionary<String> dict, BytesConverter converter) {
+        try {
+            ByteArrayOutputStream bout = new ByteArrayOutputStream();
+            DataOutputStream dataout = new DataOutputStream(bout);
+            dict.write(dataout);
+            dataout.close();
+            ByteArrayInputStream bin = new ByteArrayInputStream(bout.toByteArray());
+            DataInputStream datain = new DataInputStream(bin);
+            AppendTrieDictionary<String> r = new AppendTrieDictionary<String>();
+            r.readFields(datain);
+            datain.close();
+            return r;
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1a9bab6/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
index 3402a0c..f74df35 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import com.google.common.collect.Maps;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.kylin.common.KylinConfig;
@@ -120,17 +121,24 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
         outputValue = new Text();
         
         dictMeasures = Lists.newArrayList();
+        oldDicts = Maps.newHashMap();
+        newDicts = Maps.newHashMap();
         for (int i = 0; i < measureDescs.size(); i++) {
             MeasureDesc measureDesc = measureDescs.get(i);
             MeasureType measureType = measureDesc.getFunction().getMeasureType();
-            if (measureType.getColumnsNeedDictionary(measureDesc.getFunction()).isEmpty() == false) {
+            List<TblColRef> columns = measureType.getColumnsNeedDictionary(measureDesc.getFunction());
+            boolean needReEncode = false;
+            for (TblColRef col : columns) {
+                if (!sourceCubeSegment.getDictionary(col).equals(mergedCubeSegment.getDictionary(col))) {
+                    oldDicts.put(col, sourceCubeSegment.getDictionary(col));
+                    newDicts.put(col, mergedCubeSegment.getDictionary(col));
+                    needReEncode = true;
+                }
+            }
+            if (needReEncode) {
                 dictMeasures.add(Pair.newPair(i, measureType.newIngester()));
             }
         }
-        if (dictMeasures.size() > 0) {
-            oldDicts = sourceCubeSegment.buildDictionaryMap();
-            newDicts = mergedCubeSegment.buildDictionaryMap();
-        }
     }
 
     private static final Pattern JOB_NAME_PATTERN = Pattern.compile("kylin-([0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12})");


[5/6] kylin git commit: KYLIN-1379 More stable and functional precise count distinct implements after KYLIN-1186

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/3032bbbe/examples/test_case_data/localmeta/data/flatten_data_for_without_slr_left_join.csv
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/data/flatten_data_for_without_slr_left_join.csv b/examples/test_case_data/localmeta/data/flatten_data_for_without_slr_left_join.csv
index ab513a5..caa8754 100644
--- a/examples/test_case_data/localmeta/data/flatten_data_for_without_slr_left_join.csv
+++ b/examples/test_case_data/localmeta/data/flatten_data_for_without_slr_left_join.csv
@@ -1,402 +1,402 @@
-2013-03-31,48028,0,\N,\N,\N,Auction,12,184.21,10000001,1
-2013-11-12,164262,0,\N,\N,\N,Others,5,172.03,10000002,1
-2013-04-06,82494,15,BookMagazines,NULL,Comic Books,Auction,14,66.6,10000003,1
-2013-05-17,66767,15,Home & Garden,NULL,Dogs,Auction,12,92.98,10000004,1
-2013-05-20,152801,0,Jewelry & Watches,NULL,Earrings,FP-GTC,5,132.33,10000005,1
-2013-06-16,43398,0,Home & Garden,NULL,Cheese & Crackers,FP-GTC,13,7.12,10000006,1
-2013-06-14,95173,0,Health & Beauty,Bath & Body,Bath Sets & Kits,Auction,14,204.28,10000007,1
-2013-03-22,158666,15,ToyHobbies,Action Figures,Anime & Manga,Auction,13,35.72,10000008,1
-2013-03-10,12688,0,eBay Premier,Books & Manuscripts,Books: Other,Auction,12,4.13,10000009,1
-2013-11-01,103324,15,ClothinShoeAccessories,Women's Shoes,Mixed Items,FP-GTC,5,27.48,10000010,1
-2013-06-16,108782,15,Vehicle Parts & Accessories,CaTruck Parts,Car Care & Cleaning,FP-GTC,14,9.26,10000011,1
-2013-09-12,80287,0,Computers/Tablets & Networking,Software,Office & Business,Auction,12,3.18,10000012,1
-2013-09-28,140746,100,eBay Motors,Parts & Accessories,Vintage Car & Truck Parts,Others,13,3.18,10000013,1
-2013-06-15,87118,0,Sporting Goods,Outdoor Sports,Paintball,ABIN,14,377.94,10000014,1
-2013-03-14,25147,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Baseball-MLB,Auction,12,146.33,10000015,1
-2013-09-01,170302,15,Crafts,Embroidery,Design CDs,FP-GTC,5,51.23,10000016,1
-2013-05-29,53064,0,Business & Industrial,Heavy Equipment,Antique & Vintage Farm Equip,FP-non GTC,13,72.65,10000017,1
-2013-05-31,132939,0,Jewelry & Watches,Fashion Jewelry,Other,Auction,13,66.6,10000018,1
-2013-03-18,113593,15,Phones,Mobile Phones,Mobile Phones,Auction,12,9.26,10000019,1
-2013-07-19,34273,100,eBay Motors,Parts & Accessories,Motorcycle,Auction,14,583.44,10000020,1
-2013-06-23,106340,15,Home & Garden,Gardening,Hand Tools,FP-GTC,14,638.72,10000021,1
-2013-05-20,150265,15,Baby,Baby Clothing,Boys,FP-GTC,14,4.54,10000022,1
-2013-05-17,24760,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Hockey-NHL,FP-GTC,12,319.79,10000023,1
-2013-03-11,37831,0,Collectibles,Advertising,Merchandise & Memorabilia,Auction,12,20.35,10000024,1
-2013-01-30,1120,3,Books,First Editions,Other,FP-non GTC,5,223.63,10000025,1
-2013-01-26,43972,100,eBay Motors,Parts & Accessories,ATV Parts,FP-GTC,13,204.28,10000026,1
-2013-03-22,166013,15,Computers,Computer Components & Parts,Video Capture & TV Tuner Cards,Auction,14,5.48,10000027,1
-2013-07-23,15568,15,Baby,Baby Clothing,Unisex,Auction,14,27.48,10000028,1
-2013-07-27,103178,15,ClothinShoeAccessories,Women's Bags,Women's Bags,FP-GTC,5,21.72,10000029,1
-2013-10-29,2023,0,Sporting Goods,Team Sports,Basketball,ABIN,12,3.18,10000030,1
-2013-10-08,94847,0,Consumer Electronics,Vehicle Electronics & GPS,Car Video,FP-GTC,11,491.32,10000031,1
-2013-04-26,15868,0,Real Estate,Land,Land,Auction,14,448.8,10000032,1
-2013-01-01,32876,0,Home & Garden,Home Improvement,Plumbing & Fixtures,Auction,13,415.73,10000033,1
-2013-01-15,62179,0,ClothinShoes & Accessories,Women's Clothing,Athletic Apparel,Auction,13,377.94,10000034,1
-2013-05-27,33038,15,Musical Instruments,Instruments,Guitars (Electric),FP-GTC,14,146.33,10000035,1
-2013-11-11,156614,0,Toys & Hobbies,Diecast & Toy Vehicles,Cars: RacinNASCAR,FP-GTC,5,7.12,10000036,1
-2013-03-08,106246,0,Health & Beauty,Hair Care & Styling,Shampoo & Conditioning,Auction,13,42.99,10000037,1
-2013-03-25,20865,0,ClothinShoes & Accessories,Men's Clothing,Athletic Apparel,Auction,13,12.85,10000038,1
-2013-08-20,15115,0,Video Games & Consoles,Video Games,Video Games,FP-GTC,13,55.89,10000039,1
-2013-05-17,3838,0,Jewelry & Watches,Fashion Jewelry,Charms & Charm Bracelets,FP-GTC,14,73.26,10000040,1
-2013-06-05,759,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,Auction,11,112.56,10000041,1
-2013-10-08,61323,0,Consumer Electronics,TVideo & Home Audio,TVideo & Audio Accessories,FP-non GTC,11,3.49,10000042,1
-2013-08-14,121153,0,Baby,Nursery Decor,Night Lights,Auction,13,184.21,10000043,1
-2013-08-14,88750,0,Consumer Electronics,Vehicle Electronics & GPS,Radar & Laser Detectors,Auction,13,157.14,10000044,1
-2013-05-17,161567,15,Computers,Laptop & Desktop Accessories,Laptop Batteries,FP-GTC,14,72.65,10000045,1
-2013-08-09,113802,15,Lots More...,Metaphysical,Herbs,FP-GTC,14,51.23,10000046,1
-2013-06-30,15808,15,ClothinShoeAccessories,Women's Clothing,Tops & Blouses,FP-non GTC,14,15.85,10000047,1
-2013-06-03,174053,3,Vehicle Parts & Accessories,Car Parts,External & Body Parts,FP-GTC,13,7.12,10000048,1
-2013-12-31,2635,0,Toys & Hobbies,Toy Soldiers,1970-Now,Auction,14,12.04,10000049,1
-2013-12-25,1161,3,DVFilm & TV,Other Formats,Videos: NTSC  (US),Auction,13,73.26,10000050,1
-2013-03-28,64076,0,Computers/Tablets & Networking,Enterprise NetworkinServers,Switches & Hubs,FP-non GTC,5,184.21,10000051,1
-2013-01-30,33977,15,Crafts,Scrapbooking,Albums,FP-GTC,13,172.03,10000052,1
-2013-12-05,31673,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Racing-NASCAR,FP-GTC,14,122.78,10000053,1
-2013-10-08,174106,3,Vehicle Parts & Accessories,Car Parts,Transmission & Drivetrain,Auction,14,92.98,10000054,1
-2013-12-27,26249,0,Business & Industrial,Printing & Graphic Arts,Commercial Printing Presses,Auction,13,12.19,10000055,1
-2013-12-16,159184,0,Sporting Goods,Winter Sports,Snowboarding,FP-GTC,5,15.65,10000056,1
-2013-10-17,10058,3,Events Tickets,Other Tickets,Other Tickets,FP-GTC,11,101.79,10000057,1
-2013-11-17,48904,0,ClothinShoes & Accessories,Vintage,Women's Vintage Clothing,ABIN,12,7.12,10000058,1
-2013-09-18,145970,0,Toys & Hobbies,Models & Kits,Automotive,FP-non GTC,14,12.85,10000059,1
-2013-06-30,963,0,ClothinShoes & Accessories,Vintage,Women's Vintage Shoes,FP-GTC,13,12.19,10000060,1
-2013-10-12,118687,3,Health & Beauty,Fragrances,Women's Fragrances,FP-GTC,13,92.98,10000061,1
-2013-08-20,20886,0,Toys & Hobbies,Diecast & Toy Vehicles,Cars: RacinNASCAR,FP-GTC,14,42.99,10000062,1
-2013-08-29,148324,15,Phones,Mobile Accessories,CaseCoverSkins,Auction,13,1.88,10000063,1
-2013-07-17,139255,15,Jewellery & Watches,Fine Jewellery,Earrings,Auction,14,21.14,10000064,1
-2013-07-23,20213,0,Collectibles,Postcards,US StateCities & Towns,FP-GTC,5,21.14,10000065,1
-2013-01-06,32996,15,Movies,Television Memorabilia,Clippings,Auction,13,132.33,10000066,1
-2013-08-14,99985,0,Collectibles,Trading Cards,Sci-FFantasy,FP-GTC,14,120.87,10000067,1
-2013-08-10,67703,3,Jewellery & Watches,Jewellery Boxes & Supplies,Jewellery Display,Auction,14,120.87,10000068,1
-2013-09-28,65,0,Collectibles,Comics,Platinum Age (1897-1937),FP-non GTC,11,9.26,10000069,1
-2013-08-21,130,0,Collectibles,Transportation,Railroadiana & Trains,FP-non GTC,14,16.26,10000070,1
-2013-03-11,164,0,Computers/Tablets & Networking,Computer Components & Parts,CPUProcessors,FP-GTC,14,157.14,10000071,1
-2013-05-06,216,0,Sports MeCards & Fan Shop,Cards,Hockey,FP-GTC,11,1.88,10000072,1
-2013-05-17,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,12.04,10000073,1
-2013-01-10,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,14,189.23,10000074,1
-2013-05-05,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,73.26,10000075,1
-2013-02-03,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,4.13,10000076,1
-2013-11-26,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,290.72,10000077,1
-2013-08-30,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,265.56,10000078,1
-2013-04-26,279,15,BookMagazines,Children's Books,Children's Books,FP-GTC,5,5.91,10000079,1
-2013-06-30,314,0,ClothinShoes & Accessories,Women's Clothing,Other,Auction,5,319.79,10000080,1
-2013-06-30,314,211,ClothinShoes & Accessories,Womens' Clothing,Other,Auction,5,246,10000081,1
-2013-12-16,314,211,ClothinShoes & Accessories,Womens' Clothing,Other,Auction,5,20.35,10000082,1
-2013-12-15,314,0,ClothinShoes & Accessories,Women's Clothing,Other,Auction,5,36.7,10000083,1
-2013-08-17,533,0,Coins & Paper Money,Coins: World,Africa,Auction,13,101.79,10000084,1
-2013-12-15,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,5,47.71,10000085,1
-2013-02-04,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,5,3.49,10000086,1
-2013-01-11,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,13,46.44,10000087,1
-2013-02-04,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,13,4.54,10000088,1
-2013-05-17,1357,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,14,3.18,10000089,1
-2013-11-12,1504,0,Business & Industrial,Electrical & Test Equipment,Test Equipment,FP-GTC,14,86.58,10000090,1
-2013-08-21,4943,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-GTC,13,12.85,10000091,1
-2013-05-09,6762,0,Unknown,Unknown,Unknown,ABIN,13,16.26,10000092,1
-2013-09-19,9426,3,Mobile Phones & Communication,Home Phones & Accessories,Phone Accessories,Auction,13,21.14,10000093,1
-2013-02-06,10866,0,Collectibles,Animals,Farm & Countryside,FP-non GTC,14,20.6,10000094,1
-2013-02-02,11554,0,ClothinShoes & Accessories,Women's Clothing,Jeans,Auction,13,246,10000095,1
-2013-08-23,11848,0,Health & Beauty,Fragrances,Women,FP-GTC,14,109,10000096,1
-2013-08-03,13836,0,Collectibles,Decorative Collectibles,Spoons,Auction,13,39.41,10000097,1
-2013-05-17,13836,0,Collectibles,Decorative Collectibles,Spoons,Auction,14,16.26,10000098,1
-2013-06-06,13987,0,Collectibles,Paper,Booklets,FP-GTC,13,112.56,10000099,1
-2013-07-02,15687,0,ClothinShoes & Accessories,Men's Clothing,T-Shirts,Auction,14,184.21,10000100,1
-2013-10-25,15687,0,ClothinShoes & Accessories,Men's Clothing,T-Shirts,Auction,11,27.48,10000001,1
-2013-04-20,16145,3,Computers/Tablets & Networking,Computer Components & Parts,Other Components & Parts,FP-non GTC,12,26.45,10000002,1
-2013-03-12,16145,0,Computers/Tablets & Networking,Computer Components & Parts,Other,FP-non GTC,13,415.73,10000003,1
-2013-03-28,16509,0,Toys & Hobbies,Model Railroads & Trains,S Scale,ABIN,5,56.36,10000004,1
-2013-10-29,16509,0,Toys & Hobbies,Model Railroads & Trains,S Scale,ABIN,5,2.44,10000005,1
-2013-05-22,20485,0,Home & Garden,Furniture,Other,FP-GTC,14,269.76,10000006,1
-2013-01-25,20485,101,CasArredamento e Bricolage,Cucina,Altro per cucina,FP-GTC,12,109,10000007,1
-2013-06-12,20485,101,CasArredamento e Bricolage,Cucina,Altro per cucina,FP-GTC,12,101.79,10000008,1
-2013-12-26,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,246,10000009,1
-2013-12-26,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,189.23,10000010,1
-2013-12-31,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,15.65,10000011,1
-2013-10-04,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,28.23,10000012,1
-2013-03-16,24541,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,College-NCAA,FP-GTC,5,16.26,10000013,1
-2013-05-21,26262,0,Collectibles,Advertising,Food & Beverage,FP-GTC,5,122.78,10000014,1
-2013-01-28,30059,3,Cameras & Photography,Lenses & Filters,Lens AdapterMounts & Tubes,FP-GTC,14,172.03,10000015,1
-2013-04-26,31387,3,Jewellery & Watches,Watches,Wristwatches,Auction,14,42.99,10000016,1
-2013-10-06,31387,3,Jewellery & Watches,Watches,Wristwatches,Auction,14,207.5,10000017,1
-2013-11-06,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,5.91,10000018,1
-2013-10-06,31519,3,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,39.41,10000019,1
-2013-12-28,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,16.26,10000020,1
-2013-11-06,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,16.26,10000021,1
-2013-11-06,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,78.48,10000022,1
-2013-12-28,31519,3,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,190.22,10000023,1
-2013-06-11,35570,100,eBay Motors,Parts & Accessories,Motorcycle Parts,FP-GTC,12,2.44,10000024,1
-2013-01-10,36250,0,Sporting Goods,Hunting,Decoys,Auction,5,7.12,10000025,1
-2013-09-17,38238,0,Home & Garden,Home Decor,Other,FP-non GTC,14,36.7,10000026,1
-2013-08-14,40059,3,Mobile Phones & Communication,Radio Communication Equipment,Parts & Accessories,FP-GTC,14,35.72,10000027,1
-2013-08-09,40059,3,Mobile Phones & Communication,Radio Communication Equipment,Parts & Accessories,FP-GTC,14,3.49,10000028,1
-2013-12-02,41940,0,Business & Industrial,Manufacturing & Metalworking,Metalworking Tooling,FP-GTC,13,223.63,10000029,1
-2013-02-01,41940,0,Business & Industrial,Manufacturing & Metalworking,Metalworking Tooling,FP-GTC,13,265.56,10000030,1
-2013-07-28,43479,0,Cameras & Photo,Film Photography,Other,FP-non GTC,13,62.02,10000031,1
-2013-06-16,44079,0,Sporting Goods,Exercise & Fitness,GyWorkout & Yoga,FP-GTC,12,46.44,10000032,1
-2013-08-23,45238,101,Abbigliamento e accessori,Donna: Accessori,SciarpFoulard e Scialli,Auction,14,132.33,10000033,1
-2013-06-15,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,Auction,13,448.8,10000034,1
-2013-06-15,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,FP-non GTC,14,207.5,10000035,1
-2013-06-01,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,FP-non GTC,14,190.22,10000036,1
-2013-08-10,46575,0,Business & Industrial,Light Equipment & Tools,Air Tools,FP-GTC,14,16.71,10000037,1
-2013-03-22,50508,0,Cameras & Photo,Camera & Photo Accessories,LCD Hoods,FP-non GTC,13,4.13,10000038,1
-2013-12-25,50508,0,Cameras & Photo,Camera & Photo Accessories,LCD Hoods,FP-non GTC,13,1.88,10000039,1
-2013-07-22,50677,0,Jewelry & Watches,Fashion Jewelry,Pins & Brooches,FP-GTC,13,491.32,10000040,1
-2013-04-13,50677,0,Jewelry & Watches,Fashion Jewelry,Pins & Brooches,FP-GTC,5,2.44,10000041,1
-2013-04-16,51582,0,ClothinShoes & Accessories,Kids' ClothinShoes & Accs,Girls' Clothing (Sizes 4 & Up),Auction,14,56.36,10000042,1
-2013-08-21,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-GTC,13,15.85,10000043,1
-2013-04-22,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-non GTC,14,2.44,10000044,1
-2013-08-29,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-GTC,14,7.12,10000045,1
-2013-05-16,57784,0,ClothinShoes & Accessories,Baby & Toddler Clothing,Boys' Clothing (Newborn-5T),Auction,14,35.72,10000046,1
-2013-08-23,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,11,9.26,10000047,1
-2013-07-10,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,3.18,10000048,1
-2013-08-10,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,638.72,10000049,1
-2013-08-23,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,141.7,10000050,1
-2013-04-18,57990,0,ClothinShoes & Accessories,Men's Clothing,Casual Shirts,ABIN,13,12.19,10000051,1
-2013-07-10,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,132.33,10000052,1
-2013-06-16,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,ABIN,5,5.48,10000053,1
-2013-07-15,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,11,1.88,10000054,1
-2013-11-06,60340,0,Entertainment Memorabilia,Movie Memorabilia,Pressbooks,FP-GTC,14,12.85,10000055,1
-2013-12-27,60340,0,Entertainment Memorabilia,Movie Memorabilia,Pressbooks,FP-GTC,14,62.02,10000056,1
-2013-07-29,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,15.85,10000057,1
-2013-11-17,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,9.26,10000058,1
-2013-07-27,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,16.71,10000059,1
-2013-07-29,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,20.6,10000060,1
-2013-01-09,63861,3,ClotheShoes & Accessories,Women's Clothing,Dresses,Auction,5,1.88,10000061,1
-2013-06-11,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,5,141.7,10000062,1
-2013-01-10,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,5,1.88,10000063,1
-2013-09-16,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Others,11,112.56,10000064,1
-2013-01-14,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Others,11,94.45,10000065,1
-2013-05-17,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Auction,14,78.48,10000066,1
-2013-06-05,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,13,5.48,10000067,1
-2013-05-24,63864,3,ClotheShoes & Accessories,Women's Clothing,Skirts,Auction,14,28.23,10000068,1
-2013-05-15,63889,0,ClothinShoes & Accessories,Women's Shoes,Mixed Items & Lots,Others,13,3.49,10000069,1
-2013-03-25,67698,2,Business & Industrial,Retail & Services,Jewellery Packaging & Display,FP-GTC,11,15.65,10000070,1
-2013-03-09,67698,0,Business & Industrial,Retail & Services,Jewelry Packaging & Display,FP-GTC,11,5.48,10000071,1
-2013-12-05,67698,0,Business & Industrial,Retail & Services,Jewelry Packaging & Display,FP-GTC,11,246,10000072,1
-2013-04-18,73506,0,Collectibles,Decorative Collectibles,Tea PotSets,FP-non GTC,13,122.78,10000073,1
-2013-11-01,75665,0,Home & Garden,YarGarden & Outdoor Living,Gardening Supplies,FP-GTC,14,223.63,10000074,1
-2013-05-03,75708,3,Toys & Games,Action Figures,TMovies & Video Games,ABIN,5,141.7,10000075,1
-2013-04-21,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,21.14,10000076,1
-2013-03-12,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,55.89,10000077,1
-2013-05-19,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,51.23,10000078,1
-2013-11-23,80135,0,Computers/Tablets & Networking,DriveStorage & Blank Media,Blank Media & Accessories,Auction,14,21.72,10000079,1
-2013-10-19,95672,3,ClotheShoes & Accessories,Women's Shoes,Trainers,Auction,14,204.28,10000080,1
-2013-05-18,95672,0,ClothinShoes & Accessories,Women's Shoes,Athletic,Others,11,21.14,10000081,1
-2013-02-01,100847,0,Half Books,Half Books,Half Books,Others,5,204.28,10000082,1
-2013-01-14,100847,0,Half Books,Half Books,Half Books,Others,5,122.78,10000083,1
-2013-08-05,139973,3,Video Games & Consoles,Games,Games,ABIN,14,94.45,10000084,1
-2013-05-19,139973,0,Video Games & Consoles,Video Games,Video Games,ABIN,11,86.58,10000085,1
-2013-12-01,150047,3,Crafts,Jewellery Making,Findings,Auction,14,56.36,10000086,1
-2013-12-02,150047,3,Crafts,Jewellery Making,Findings,Auction,14,290.72,10000087,1
-2013-01-11,155226,0,ClothinShoes & Accessories,Women's Clothing,Sweats & Hoodies,FP-GTC,13,60.37,10000088,1
-2013-05-27,155226,0,ClothinShoes & Accessories,Women's Clothing,Sweats & Hoodies,FP-GTC,13,112.56,10000089,1
-2013-09-01,156356,0,Collectibles,Postcards,BuildingArchitecture,FP-GTC,13,265.56,10000090,1
-2013-04-11,158798,0,Toys & Hobbies,Vintage & Antique Toys,Spinning Tops,FP-GTC,11,35.72,10000091,1
-2013-05-05,165888,0,Jewelry & Watches,Vintage & Antique Jewelry,Costume,FP-non GTC,13,92.98,10000092,1
-2013-11-21,170083,3,Computers/Tablets & Networking,Computer Components & Parts,Memory (RAM),Auction,11,28.23,10000093,1
-2013-10-07,170083,3,Computers/Tablets & Networking,Computer Components & Parts,Memory (RAM),Auction,11,27.48,10000094,1
-2013-07-12,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,9.26,10000095,1
-2013-06-07,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,3.18,10000096,1
-2013-05-22,175750,0,Home & Garden,Bedding,Blankets & Throws,FP-GTC,14,12.04,10000097,1
-2013-11-28,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,13,20.6,10000098,1
-2013-07-12,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,13,12.04,10000099,1
-2013-06-07,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,4.13,10000100,1
-2013-12-01,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,73.26,10000201,1
-2012-10-12,48027,0,Coins & Paper Money,Paper Money: World,Asia,Auction,12,184.21,10000001,1
-2012-08-20,164261,0,Jewelry & Watches,Fashion Jewelry,Earrings,Others,5,172.03,10000002,1
-2012-09-18,82494,15,BookMagazines,NULL,Comic Books,Auction,14,66.6,10000003,1
-2012-06-30,66767,15,Home & Garden,NULL,Dogs,Auction,12,92.98,10000004,1
-2012-08-29,152801,0,Jewelry & Watches,NULL,Earrings,FP-GTC,5,132.33,10000005,1
-2012-01-06,43398,0,Home & Garden,NULL,Cheese & Crackers,FP-GTC,13,7.12,10000006,1
-2012-08-14,95173,0,Health & Beauty,Bath & Body,Bath Sets & Kits,Auction,14,204.28,10000007,1
-2012-07-17,158666,15,ToyHobbies,Action Figures,Anime & Manga,Auction,13,35.72,10000008,1
-2012-07-23,12688,0,eBay Premier,Books & Manuscripts,Books: Other,Auction,12,4.13,10000009,1
-2012-07-27,103324,15,ClothinShoeAccessories,Women's Shoes,Mixed Items,FP-GTC,5,27.48,10000010,1
-2012-07-29,108782,15,Vehicle Parts & Accessories,CaTruck Parts,Car Care & Cleaning,FP-GTC,14,9.26,10000011,1
-2012-11-17,80287,0,Computers/Tablets & Networking,Software,Office & Business,Auction,12,3.18,10000012,1
-2012-10-29,140746,100,eBay Motors,Parts & Accessories,Vintage Car & Truck Parts,Others,13,3.18,10000013,1
-2012-03-28,87118,0,Sporting Goods,Outdoor Sports,Paintball,ABIN,14,377.94,10000014,1
-2012-01-30,25147,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Baseball-MLB,Auction,12,146.33,10000015,1
-2012-10-07,170302,15,Crafts,Embroidery,Design CDs,FP-GTC,5,51.23,10000016,1
-2012-11-28,53064,0,Business & Industrial,Heavy Equipment,Antique & Vintage Farm Equip,FP-non GTC,13,72.65,10000017,1
-2012-07-12,132939,0,Jewelry & Watches,Fashion Jewelry,Other,Auction,13,66.6,10000018,1
-2012-06-07,113593,15,Phones,Mobile Phones,Mobile Phones,Auction,12,9.26,10000019,1
-2012-12-01,34273,100,eBay Motors,Parts & Accessories,Motorcycle,Auction,14,583.44,10000020,1
-2012-12-02,106340,15,Home & Garden,Gardening,Hand Tools,FP-GTC,14,638.72,10000021,1
-2012-02-01,150265,15,Baby,Baby Clothing,Boys,FP-GTC,14,4.54,10000022,1
-2012-01-14,24760,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Hockey-NHL,FP-GTC,12,319.79,10000023,1
-2012-08-14,37831,0,Collectibles,Advertising,Merchandise & Memorabilia,Auction,12,20.35,10000024,1
-2012-10-04,1120,3,Books,First Editions,Other,FP-non GTC,5,223.63,10000025,1
-2012-12-26,43972,100,eBay Motors,Parts & Accessories,ATV Parts,FP-GTC,13,204.28,10000026,1
-2012-12-31,166013,15,Computers,Computer Components & Parts,Video Capture & TV Tuner Cards,Auction,14,5.48,10000027,1
-2012-03-22,15568,15,Baby,Baby Clothing,Unisex,Auction,14,27.48,10000028,1
-2012-12-25,103178,15,ClothinShoeAccessories,Women's Bags,Women's Bags,FP-GTC,5,21.72,10000029,1
-2012-12-05,2023,0,Sporting Goods,Team Sports,Basketball,ABIN,12,3.18,10000030,1
-2012-05-27,94847,0,Consumer Electronics,Vehicle Electronics & GPS,Car Video,FP-GTC,11,491.32,10000031,1
-2012-01-11,15868,0,Real Estate,Land,Land,Auction,14,448.8,10000032,1
-2012-02-04,32876,0,Home & Garden,Home Improvement,Plumbing & Fixtures,Auction,13,415.73,10000033,1
-2012-12-15,62179,0,ClothinShoes & Accessories,Women's Clothing,Athletic Apparel,Auction,13,377.94,10000034,1
-2012-06-30,33038,15,Musical Instruments,Instruments,Guitars (Electric),FP-GTC,14,146.33,10000035,1
-2012-12-16,156614,0,Toys & Hobbies,Diecast & Toy Vehicles,Cars: RacinNASCAR,FP-GTC,5,7.12,10000036,1
-2012-10-17,106246,0,Health & Beauty,Hair Care & Styling,Shampoo & Conditioning,Auction,13,42.99,10000037,1
-2012-10-08,20865,0,ClothinShoes & Accessories,Men's Clothing,Athletic Apparel,Auction,13,12.85,10000038,1
-2012-04-26,15115,0,Video Games & Consoles,Video Games,Video Games,FP-GTC,13,55.89,10000039,1
-2012-10-06,3838,0,Jewelry & Watches,Fashion Jewelry,Charms & Charm Bracelets,FP-GTC,14,73.26,10000040,1
-2012-12-28,759,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,Auction,11,112.56,10000041,1
-2012-11-06,61323,0,Consumer Electronics,TVideo & Home Audio,TVideo & Audio Accessories,FP-non GTC,11,3.49,10000042,1
-2012-11-06,121153,0,Baby,Nursery Decor,Night Lights,Auction,13,184.21,10000043,1
-2012-12-27,88750,0,Consumer Electronics,Vehicle Electronics & GPS,Radar & Laser Detectors,Auction,13,157.14,10000044,1
-2012-07-15,161567,15,Computers,Laptop & Desktop Accessories,Laptop Batteries,FP-GTC,14,72.65,10000045,1
-2012-08-23,113802,15,Lots More...,Metaphysical,Herbs,FP-GTC,14,51.23,10000046,1
-2012-07-10,15808,15,ClothinShoeAccessories,Women's Clothing,Tops & Blouses,FP-non GTC,14,15.85,10000047,1
-2012-08-10,174053,3,Vehicle Parts & Accessories,Car Parts,External & Body Parts,FP-GTC,13,7.12,10000048,1
-2012-09-01,2635,0,Toys & Hobbies,Toy Soldiers,1970-Now,Auction,14,12.04,10000049,1
-2012-09-19,1161,3,DVFilm & TV,Other Formats,Videos: NTSC  (US),Auction,13,73.26,10000050,1
-2012-08-10,64076,0,Computers/Tablets & Networking,Enterprise NetworkinServers,Switches & Hubs,FP-non GTC,5,184.21,10000051,1
-2012-08-21,33977,15,Crafts,Scrapbooking,Albums,FP-GTC,13,172.03,10000052,1
-2012-08-29,31673,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Racing-NASCAR,FP-GTC,14,122.78,10000053,1
-2012-04-22,174106,3,Vehicle Parts & Accessories,Car Parts,Transmission & Drivetrain,Auction,14,92.98,10000054,1
-2012-07-02,26249,0,Business & Industrial,Printing & Graphic Arts,Commercial Printing Presses,Auction,13,12.19,10000055,1
-2012-01-28,159184,0,Sporting Goods,Winter Sports,Snowboarding,FP-GTC,5,15.65,10000056,1
-2012-03-11,10058,3,Events Tickets,Other Tickets,Other Tickets,FP-GTC,11,101.79,10000057,1
-2012-05-29,48904,0,ClothinShoes & Accessories,Vintage,Women's Vintage Clothing,ABIN,12,7.12,10000058,1
-2012-05-31,145970,0,Toys & Hobbies,Models & Kits,Automotive,FP-non GTC,14,12.85,10000059,1
-2012-10-25,963,0,ClothinShoes & Accessories,Vintage,Women's Vintage Shoes,FP-GTC,13,12.19,10000060,1
-2012-11-23,118687,3,Health & Beauty,Fragrances,Women's Fragrances,FP-GTC,13,92.98,10000061,1
-2012-09-28,20886,0,Toys & Hobbies,Diecast & Toy Vehicles,Cars: RacinNASCAR,FP-GTC,14,42.99,10000062,1
-2012-09-28,148324,15,Phones,Mobile Accessories,CaseCoverSkins,Auction,13,1.88,10000063,1
-2012-10-19,139255,15,Jewellery & Watches,Fine Jewellery,Earrings,Auction,14,21.14,10000064,1
-2012-08-05,20213,0,Collectibles,Postcards,US StateCities & Towns,FP-GTC,5,21.14,10000065,1
-2012-05-19,32996,15,Movies,Television Memorabilia,Clippings,Auction,13,132.33,10000066,1
-2012-04-11,99985,0,Collectibles,Trading Cards,Sci-FFantasy,FP-GTC,14,120.87,10000067,1
-2012-05-16,67703,3,Jewellery & Watches,Jewellery Boxes & Supplies,Jewellery Display,Auction,14,120.87,10000068,1
-2012-05-24,65,0,Collectibles,Comics,Platinum Age (1897-1937),FP-non GTC,11,9.26,10000069,1
-2012-01-10,130,0,Collectibles,Transportation,Railroadiana & Trains,FP-non GTC,14,16.26,10000070,1
-2012-05-17,164,0,Computers/Tablets & Networking,Computer Components & Parts,CPUProcessors,FP-GTC,14,157.14,10000071,1
-2012-02-03,216,0,Sports MeCards & Fan Shop,Cards,Hockey,FP-GTC,11,1.88,10000072,1
-2012-05-05,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,12.04,10000073,1
-2012-11-26,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,14,189.23,10000074,1
-2012-08-30,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,73.26,10000075,1
-2012-05-21,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,4.13,10000076,1
-2012-06-06,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,290.72,10000077,1
-2012-06-16,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,265.56,10000078,1
-2012-05-15,279,15,BookMagazines,Children's Books,Children's Books,FP-GTC,5,5.91,10000079,1
-2012-12-05,314,0,ClothinShoes & Accessories,Women's Clothing,Other,Auction,5,319.79,10000080,1
-2012-03-25,314,211,ClothinShoes & Accessories,Womens' Clothing,Other,Auction,5,246,10000081,1
-2012-03-09,314,211,ClothinShoes & Accessories,Womens' Clothing,Other,Auction,5,20.35,10000082,1
-2012-05-06,314,0,ClothinShoes & Accessories,Women's Clothing,Other,Auction,5,36.7,10000083,1
-2012-06-15,533,0,Coins & Paper Money,Coins: World,Africa,Auction,13,101.79,10000084,1
-2012-03-14,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,5,47.71,10000085,1
-2012-05-20,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,5,3.49,10000086,1
-2012-05-17,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,13,46.44,10000087,1
-2012-03-11,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,13,4.54,10000088,1
-2012-01-10,1357,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,14,3.18,10000089,1
-2012-04-13,1504,0,Business & Industrial,Electrical & Test Equipment,Test Equipment,FP-GTC,14,86.58,10000090,1
-2012-07-22,4943,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-GTC,13,12.85,10000091,1
-2012-06-15,6762,0,Unknown,Unknown,Unknown,ABIN,13,16.26,10000092,1
-2012-06-15,9426,3,Mobile Phones & Communication,Home Phones & Accessories,Phone Accessories,Auction,13,21.14,10000093,1
-2012-06-01,10866,0,Collectibles,Animals,Farm & Countryside,FP-non GTC,14,20.6,10000094,1
-2012-05-17,11554,0,ClothinShoes & Accessories,Women's Clothing,Jeans,Auction,13,246,10000095,1
-2012-04-26,11848,0,Health & Beauty,Fragrances,Women,FP-GTC,14,109,10000096,1
-2012-07-28,13836,0,Collectibles,Decorative Collectibles,Spoons,Auction,13,39.41,10000097,1
-2012-03-12,13836,0,Collectibles,Decorative Collectibles,Spoons,Auction,14,16.26,10000098,1
-2012-04-20,13987,0,Collectibles,Paper,Booklets,FP-GTC,13,112.56,10000099,1
-2012-03-18,15687,0,ClothinShoes & Accessories,Men's Clothing,T-Shirts,Auction,14,184.21,10000100,1
-2012-05-17,15687,0,ClothinShoes & Accessories,Men's Clothing,T-Shirts,Auction,11,27.48,10000001,1
-2012-06-23,16145,3,Computers/Tablets & Networking,Computer Components & Parts,Other Components & Parts,FP-non GTC,12,26.45,10000002,1
-2012-05-22,16145,0,Computers/Tablets & Networking,Computer Components & Parts,Other,FP-non GTC,13,415.73,10000003,1
-2012-01-25,16509,0,Toys & Hobbies,Model Railroads & Trains,S Scale,ABIN,5,56.36,10000004,1
-2012-06-12,16509,0,Toys & Hobbies,Model Railroads & Trains,S Scale,ABIN,5,2.44,10000005,1
-2012-05-17,20485,0,Home & Garden,Furniture,Other,FP-GTC,14,269.76,10000006,1
-2012-08-03,20485,101,CasArredamento e Bricolage,Cucina,Altro per cucina,FP-GTC,12,109,10000007,1
-2012-05-17,20485,101,CasArredamento e Bricolage,Cucina,Altro per cucina,FP-GTC,12,101.79,10000008,1
-2012-08-21,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,246,10000009,1
-2012-08-21,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,189.23,10000010,1
-2012-04-18,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,15.65,10000011,1
-2012-06-16,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,28.23,10000012,1
-2012-01-09,24541,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,College-NCAA,FP-GTC,5,16.26,10000013,1
-2012-05-03,26262,0,Collectibles,Advertising,Food & Beverage,FP-GTC,5,122.78,10000014,1
-2012-05-20,30059,3,Cameras & Photography,Lenses & Filters,Lens AdapterMounts & Tubes,FP-GTC,14,172.03,10000015,1
-2012-03-12,31387,3,Jewellery & Watches,Watches,Wristwatches,Auction,14,42.99,10000016,1
-2012-05-19,31387,3,Jewellery & Watches,Watches,Wristwatches,Auction,14,207.5,10000017,1
-2012-04-21,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,5.91,10000018,1
-2012-05-18,31519,3,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,39.41,10000019,1
-2012-06-16,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,16.26,10000020,1
-2012-06-11,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,16.26,10000021,1
-2012-03-31,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,78.48,10000022,1
-2012-11-12,31519,3,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,190.22,10000023,1
-2012-04-06,35570,100,eBay Motors,Parts & Accessories,Motorcycle Parts,FP-GTC,12,2.44,10000024,1
-2012-11-01,36250,0,Sporting Goods,Hunting,Decoys,Auction,5,7.12,10000025,1
-2012-02-06,38238,0,Home & Garden,Home Decor,Other,FP-non GTC,14,36.7,10000026,1
-2012-06-16,40059,3,Mobile Phones & Communication,Radio Communication Equipment,Parts & Accessories,FP-GTC,14,35.72,10000027,1
-2012-09-12,40059,3,Mobile Phones & Communication,Radio Communication Equipment,Parts & Accessories,FP-GTC,14,3.49,10000028,1
-2012-04-16,41940,0,Business & Industrial,Manufacturing & Metalworking,Metalworking Tooling,FP-GTC,13,223.63,10000029,1
-2012-11-01,41940,0,Business & Industrial,Manufacturing & Metalworking,Metalworking Tooling,FP-GTC,13,265.56,10000030,1
-2012-06-14,43479,0,Cameras & Photo,Film Photography,Other,FP-non GTC,13,62.02,10000031,1
-2012-11-12,44079,0,Sporting Goods,Exercise & Fitness,GyWorkout & Yoga,FP-GTC,12,46.44,10000032,1
-2012-03-22,45238,101,Abbigliamento e accessori,Donna: Accessori,SciarpFoulard e Scialli,Auction,14,132.33,10000033,1
-2012-05-22,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,Auction,13,448.8,10000034,1
-2012-03-10,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,FP-non GTC,14,207.5,10000035,1
-2012-05-17,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,FP-non GTC,14,190.22,10000036,1
-2012-09-01,46575,0,Business & Industrial,Light Equipment & Tools,Air Tools,FP-GTC,14,16.71,10000037,1
-2012-04-18,50508,0,Cameras & Photo,Camera & Photo Accessories,LCD Hoods,FP-non GTC,13,4.13,10000038,1
-2012-06-05,50508,0,Cameras & Photo,Camera & Photo Accessories,LCD Hoods,FP-non GTC,13,1.88,10000039,1
-2012-10-08,50677,0,Jewelry & Watches,Fashion Jewelry,Pins & Brooches,FP-GTC,13,491.32,10000040,1
-2012-09-17,50677,0,Jewelry & Watches,Fashion Jewelry,Pins & Brooches,FP-GTC,5,2.44,10000041,1
-2012-02-02,51582,0,ClothinShoes & Accessories,Kids' ClothinShoes & Accs,Girls' Clothing (Sizes 4 & Up),Auction,14,56.36,10000042,1
-2012-08-23,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-GTC,13,15.85,10000043,1
-2012-08-17,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-non GTC,14,2.44,10000044,1
-2012-05-05,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-GTC,14,7.12,10000045,1
-2012-03-08,57784,0,ClothinShoes & Accessories,Baby & Toddler Clothing,Boys' Clothing (Newborn-5T),Auction,14,35.72,10000046,1
-2012-03-16,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,11,9.26,10000047,1
-2012-05-09,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,3.18,10000048,1
-2012-03-25,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,638.72,10000049,1
-2012-06-05,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,141.7,10000050,1
-2012-01-10,57990,0,ClothinShoes & Accessories,Men's Clothing,Casual Shirts,ABIN,13,12.19,10000051,1
-2012-06-11,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,132.33,10000052,1
-2012-08-23,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,ABIN,5,5.48,10000053,1
-2012-08-20,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,11,1.88,10000054,1
-2012-08-09,60340,0,Entertainment Memorabilia,Movie Memorabilia,Pressbooks,FP-GTC,14,12.85,10000055,1
-2012-06-30,60340,0,Entertainment Memorabilia,Movie Memorabilia,Pressbooks,FP-GTC,14,62.02,10000056,1
-2012-06-03,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,15.85,10000057,1
-2012-08-14,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,9.26,10000058,1
-2012-08-14,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,16.71,10000059,1
-2012-05-17,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,20.6,10000060,1
-2012-07-23,63861,3,ClotheShoes & Accessories,Women's Clothing,Dresses,Auction,5,1.88,10000061,1
-2012-07-27,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,5,141.7,10000062,1
-2012-07-29,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,5,1.88,10000063,1
-2012-11-17,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Others,11,112.56,10000064,1
-2012-10-29,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Others,11,94.45,10000065,1
-2012-03-28,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Auction,14,78.48,10000066,1
-2012-01-30,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,13,5.48,10000067,1
-2012-01-26,63864,3,ClotheShoes & Accessories,Women's Clothing,Skirts,Auction,14,28.23,10000068,1
-2012-11-21,63889,0,ClothinShoes & Accessories,Women's Shoes,Mixed Items & Lots,Others,13,3.49,10000069,1
-2012-07-12,67698,2,Business & Industrial,Retail & Services,Jewellery Packaging & Display,FP-GTC,11,15.65,10000070,1
-2012-06-07,67698,0,Business & Industrial,Retail & Services,Jewelry Packaging & Display,FP-GTC,11,5.48,10000071,1
-2012-12-01,67698,0,Business & Industrial,Retail & Services,Jewelry Packaging & Display,FP-GTC,11,246,10000072,1
-2012-12-02,73506,0,Collectibles,Decorative Collectibles,Tea PotSets,FP-non GTC,13,122.78,10000073,1
-2012-02-01,75665,0,Home & Garden,YarGarden & Outdoor Living,Gardening Supplies,FP-GTC,14,223.63,10000074,1
-2012-01-14,75708,3,Toys & Games,Action Figures,TMovies & Video Games,ABIN,5,141.7,10000075,1
-2012-09-16,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,21.14,10000076,1
-2012-08-09,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,55.89,10000077,1
-2012-12-26,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,51.23,10000078,1
-2012-12-31,80135,0,Computers/Tablets & Networking,DriveStorage & Blank Media,Blank Media & Accessories,Auction,14,21.72,10000079,1
-2012-03-22,95672,3,ClotheShoes & Accessories,Women's Shoes,Trainers,Auction,14,204.28,10000080,1
-2012-12-25,95672,0,ClothinShoes & Accessories,Women's Shoes,Athletic,Others,11,21.14,10000081,1
-2012-01-15,100847,0,Half Books,Half Books,Half Books,Others,5,204.28,10000082,1
-2012-05-27,100847,0,Half Books,Half Books,Half Books,Others,5,122.78,10000083,1
-2012-01-11,139973,3,Video Games & Consoles,Games,Games,ABIN,14,94.45,10000084,1
-2012-02-04,139973,0,Video Games & Consoles,Video Games,Video Games,ABIN,11,86.58,10000085,1
-2012-12-15,150047,3,Crafts,Jewellery Making,Findings,Auction,14,56.36,10000086,1
-2012-06-30,150047,3,Crafts,Jewellery Making,Findings,Auction,14,290.72,10000087,1
-2012-12-16,155226,0,ClothinShoes & Accessories,Women's Clothing,Sweats & Hoodies,FP-GTC,13,60.37,10000088,1
-2012-11-11,155226,0,ClothinShoes & Accessories,Women's Clothing,Sweats & Hoodies,FP-GTC,13,112.56,10000089,1
-2012-10-08,156356,0,Collectibles,Postcards,BuildingArchitecture,FP-GTC,13,265.56,10000090,1
-2012-04-26,158798,0,Toys & Hobbies,Vintage & Antique Toys,Spinning Tops,FP-GTC,11,35.72,10000091,1
-2012-10-06,165888,0,Jewelry & Watches,Vintage & Antique Jewelry,Costume,FP-non GTC,13,92.98,10000092,1
-2012-12-28,170083,3,Computers/Tablets & Networking,Computer Components & Parts,Memory (RAM),Auction,11,28.23,10000093,1
-2012-11-06,170083,3,Computers/Tablets & Networking,Computer Components & Parts,Memory (RAM),Auction,11,27.48,10000094,1
-2012-11-06,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,9.26,10000095,1
-2012-12-27,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,3.18,10000096,1
-2012-01-01,175750,0,Home & Garden,Bedding,Blankets & Throws,FP-GTC,14,12.04,10000097,1
-2012-08-23,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,13,20.6,10000098,1
-2012-07-10,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,13,12.04,10000099,1
-2012-08-10,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,4.13,10000100,1
-2012-07-19,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,73.26,10000201,1
+2013-03-31,48028,0,\N,\N,\N,Auction,12,184.21,10000001,1,Ebay
+2013-11-12,164262,0,\N,\N,\N,Others,5,172.03,10000002,1,Ebay
+2013-04-06,82494,15,BookMagazines,NULL,Comic Books,Auction,14,66.6,10000003,1,Ebay
+2013-05-17,66767,15,Home & Garden,NULL,Dogs,Auction,12,92.98,10000004,1,Ebay
+2013-05-20,152801,0,Jewelry & Watches,NULL,Earrings,FP-GTC,5,132.33,10000005,1,Ebay
+2013-06-16,43398,0,Home & Garden,NULL,Cheese & Crackers,FP-GTC,13,7.12,10000006,1,Ebay
+2013-06-14,95173,0,Health & Beauty,Bath & Body,Bath Sets & Kits,Auction,14,204.28,10000007,1,Ebay
+2013-03-22,158666,15,ToyHobbies,Action Figures,Anime & Manga,Auction,13,35.72,10000008,1,Ebay
+2013-03-10,12688,0,eBay Premier,Books & Manuscripts,Books: Other,Auction,12,4.13,10000009,1,Ebay
+2013-11-01,103324,15,ClothinShoeAccessories,Women's Shoes,Mixed Items,FP-GTC,5,27.48,10000010,1,Ebay
+2013-06-16,108782,15,Vehicle Parts & Accessories,CaTruck Parts,Car Care & Cleaning,FP-GTC,14,9.26,10000011,1,Ebay
+2013-09-12,80287,0,Computers/Tablets & Networking,Software,Office & Business,Auction,12,3.18,10000012,1,Ebay
+2013-09-28,140746,100,eBay Motors,Parts & Accessories,Vintage Car & Truck Parts,Others,13,3.18,10000013,1,Ebay
+2013-06-15,87118,0,Sporting Goods,Outdoor Sports,Paintball,ABIN,14,377.94,10000014,1,Ebay
+2013-03-14,25147,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Baseball-MLB,Auction,12,146.33,10000015,1,Ebay
+2013-09-01,170302,15,Crafts,Embroidery,Design CDs,FP-GTC,5,51.23,10000016,1,Ebay
+2013-05-29,53064,0,Business & Industrial,Heavy Equipment,Antique & Vintage Farm Equip,FP-non GTC,13,72.65,10000017,1,Ebay
+2013-05-31,132939,0,Jewelry & Watches,Fashion Jewelry,Other,Auction,13,66.6,10000018,1,Ebay
+2013-03-18,113593,15,Phones,Mobile Phones,Mobile Phones,Auction,12,9.26,10000019,1,Ebay
+2013-07-19,34273,100,eBay Motors,Parts & Accessories,Motorcycle,Auction,14,583.44,10000020,1,Ebay
+2013-06-23,106340,15,Home & Garden,Gardening,Hand Tools,FP-GTC,14,638.72,10000021,1,Ebay
+2013-05-20,150265,15,Baby,Baby Clothing,Boys,FP-GTC,14,4.54,10000022,1,Ebay
+2013-05-17,24760,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Hockey-NHL,FP-GTC,12,319.79,10000023,1,Ebay
+2013-03-11,37831,0,Collectibles,Advertising,Merchandise & Memorabilia,Auction,12,20.35,10000024,1,Ebay
+2013-01-30,1120,3,Books,First Editions,Other,FP-non GTC,5,223.63,10000025,1,Ebay
+2013-01-26,43972,100,eBay Motors,Parts & Accessories,ATV Parts,FP-GTC,13,204.28,10000026,1,Ebay
+2013-03-22,166013,15,Computers,Computer Components & Parts,Video Capture & TV Tuner Cards,Auction,14,5.48,10000027,1,Ebay
+2013-07-23,15568,15,Baby,Baby Clothing,Unisex,Auction,14,27.48,10000028,1,Ebay
+2013-07-27,103178,15,ClothinShoeAccessories,Women's Bags,Women's Bags,FP-GTC,5,21.72,10000029,1,Ebay
+2013-10-29,2023,0,Sporting Goods,Team Sports,Basketball,ABIN,12,3.18,10000030,1,Ebay
+2013-10-08,94847,0,Consumer Electronics,Vehicle Electronics & GPS,Car Video,FP-GTC,11,491.32,10000031,1,Ebay
+2013-04-26,15868,0,Real Estate,Land,Land,Auction,14,448.8,10000032,1,Ebay
+2013-01-01,32876,0,Home & Garden,Home Improvement,Plumbing & Fixtures,Auction,13,415.73,10000033,1,Ebay
+2013-01-15,62179,0,ClothinShoes & Accessories,Women's Clothing,Athletic Apparel,Auction,13,377.94,10000034,1,Ebay
+2013-05-27,33038,15,Musical Instruments,Instruments,Guitars (Electric),FP-GTC,14,146.33,10000035,1,Ebay
+2013-11-11,156614,0,Toys & Hobbies,Diecast & Toy Vehicles,Cars: RacinNASCAR,FP-GTC,5,7.12,10000036,1,Ebay
+2013-03-08,106246,0,Health & Beauty,Hair Care & Styling,Shampoo & Conditioning,Auction,13,42.99,10000037,1,Ebay
+2013-03-25,20865,0,ClothinShoes & Accessories,Men's Clothing,Athletic Apparel,Auction,13,12.85,10000038,1,Ebay
+2013-08-20,15115,0,Video Games & Consoles,Video Games,Video Games,FP-GTC,13,55.89,10000039,1,Ebay
+2013-05-17,3838,0,Jewelry & Watches,Fashion Jewelry,Charms & Charm Bracelets,FP-GTC,14,73.26,10000040,1,Ebay
+2013-06-05,759,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,Auction,11,112.56,10000041,1,Ebay
+2013-10-08,61323,0,Consumer Electronics,TVideo & Home Audio,TVideo & Audio Accessories,FP-non GTC,11,3.49,10000042,1,Ebay
+2013-08-14,121153,0,Baby,Nursery Decor,Night Lights,Auction,13,184.21,10000043,1,Ebay
+2013-08-14,88750,0,Consumer Electronics,Vehicle Electronics & GPS,Radar & Laser Detectors,Auction,13,157.14,10000044,1,Ebay
+2013-05-17,161567,15,Computers,Laptop & Desktop Accessories,Laptop Batteries,FP-GTC,14,72.65,10000045,1,Ebay
+2013-08-09,113802,15,Lots More...,Metaphysical,Herbs,FP-GTC,14,51.23,10000046,1,Ebay
+2013-06-30,15808,15,ClothinShoeAccessories,Women's Clothing,Tops & Blouses,FP-non GTC,14,15.85,10000047,1,Ebay
+2013-06-03,174053,3,Vehicle Parts & Accessories,Car Parts,External & Body Parts,FP-GTC,13,7.12,10000048,1,Ebay
+2013-12-31,2635,0,Toys & Hobbies,Toy Soldiers,1970-Now,Auction,14,12.04,10000049,1,Ebay
+2013-12-25,1161,3,DVFilm & TV,Other Formats,Videos: NTSC  (US),Auction,13,73.26,10000050,1,Ebay
+2013-03-28,64076,0,Computers/Tablets & Networking,Enterprise NetworkinServers,Switches & Hubs,FP-non GTC,5,184.21,10000051,1,Ebay
+2013-01-30,33977,15,Crafts,Scrapbooking,Albums,FP-GTC,13,172.03,10000052,1,Ebay
+2013-12-05,31673,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Racing-NASCAR,FP-GTC,14,122.78,10000053,1,Ebay
+2013-10-08,174106,3,Vehicle Parts & Accessories,Car Parts,Transmission & Drivetrain,Auction,14,92.98,10000054,1,Ebay
+2013-12-27,26249,0,Business & Industrial,Printing & Graphic Arts,Commercial Printing Presses,Auction,13,12.19,10000055,1,Ebay
+2013-12-16,159184,0,Sporting Goods,Winter Sports,Snowboarding,FP-GTC,5,15.65,10000056,1,Ebay
+2013-10-17,10058,3,Events Tickets,Other Tickets,Other Tickets,FP-GTC,11,101.79,10000057,1,Ebay
+2013-11-17,48904,0,ClothinShoes & Accessories,Vintage,Women's Vintage Clothing,ABIN,12,7.12,10000058,1,Ebay
+2013-09-18,145970,0,Toys & Hobbies,Models & Kits,Automotive,FP-non GTC,14,12.85,10000059,1,Ebay
+2013-06-30,963,0,ClothinShoes & Accessories,Vintage,Women's Vintage Shoes,FP-GTC,13,12.19,10000060,1,Ebay
+2013-10-12,118687,3,Health & Beauty,Fragrances,Women's Fragrances,FP-GTC,13,92.98,10000061,1,Ebay
+2013-08-20,20886,0,Toys & Hobbies,Diecast & Toy Vehicles,Cars: RacinNASCAR,FP-GTC,14,42.99,10000062,1,Ebay
+2013-08-29,148324,15,Phones,Mobile Accessories,CaseCoverSkins,Auction,13,1.88,10000063,1,Ebay
+2013-07-17,139255,15,Jewellery & Watches,Fine Jewellery,Earrings,Auction,14,21.14,10000064,1,Ebay
+2013-07-23,20213,0,Collectibles,Postcards,US StateCities & Towns,FP-GTC,5,21.14,10000065,1,Ebay
+2013-01-06,32996,15,Movies,Television Memorabilia,Clippings,Auction,13,132.33,10000066,1,Ebay
+2013-08-14,99985,0,Collectibles,Trading Cards,Sci-FFantasy,FP-GTC,14,120.87,10000067,1,Ebay
+2013-08-10,67703,3,Jewellery & Watches,Jewellery Boxes & Supplies,Jewellery Display,Auction,14,120.87,10000068,1,Ebay
+2013-09-28,65,0,Collectibles,Comics,Platinum Age (1897-1937),FP-non GTC,11,9.26,10000069,1,Ebay
+2013-08-21,130,0,Collectibles,Transportation,Railroadiana & Trains,FP-non GTC,14,16.26,10000070,1,Ebay
+2013-03-11,164,0,Computers/Tablets & Networking,Computer Components & Parts,CPUProcessors,FP-GTC,14,157.14,10000071,1,Ebay
+2013-05-06,216,0,Sports MeCards & Fan Shop,Cards,Hockey,FP-GTC,11,1.88,10000072,1,Ebay
+2013-05-17,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,12.04,10000073,1,Ebay
+2013-01-10,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,14,189.23,10000074,1,Ebay
+2013-05-05,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,73.26,10000075,1,Ebay
+2013-02-03,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,4.13,10000076,1,Ebay
+2013-11-26,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,290.72,10000077,1,Ebay
+2013-08-30,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,265.56,10000078,1,Ebay
+2013-04-26,279,15,BookMagazines,Children's Books,Children's Books,FP-GTC,5,5.91,10000079,1,Ebay
+2013-06-30,314,0,ClothinShoes & Accessories,Women's Clothing,Other,Auction,5,319.79,10000080,1,Ebay
+2013-06-30,314,211,ClothinShoes & Accessories,Womens' Clothing,Other,Auction,5,246,10000081,1,Ebay
+2013-12-16,314,211,ClothinShoes & Accessories,Womens' Clothing,Other,Auction,5,20.35,10000082,1,Ebay
+2013-12-15,314,0,ClothinShoes & Accessories,Women's Clothing,Other,Auction,5,36.7,10000083,1,Ebay
+2013-08-17,533,0,Coins & Paper Money,Coins: World,Africa,Auction,13,101.79,10000084,1,Ebay
+2013-12-15,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,5,47.71,10000085,1,Ebay
+2013-02-04,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,5,3.49,10000086,1,Ebay
+2013-01-11,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,13,46.44,10000087,1,Ebay
+2013-02-04,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,13,4.54,10000088,1,Ebay
+2013-05-17,1357,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,14,3.18,10000089,1,Ebay
+2013-11-12,1504,0,Business & Industrial,Electrical & Test Equipment,Test Equipment,FP-GTC,14,86.58,10000090,1,Ebay
+2013-08-21,4943,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-GTC,13,12.85,10000091,1,Ebay
+2013-05-09,6762,0,Unknown,Unknown,Unknown,ABIN,13,16.26,10000092,1,Ebay
+2013-09-19,9426,3,Mobile Phones & Communication,Home Phones & Accessories,Phone Accessories,Auction,13,21.14,10000093,1,Ebay
+2013-02-06,10866,0,Collectibles,Animals,Farm & Countryside,FP-non GTC,14,20.6,10000094,1,Ebay
+2013-02-02,11554,0,ClothinShoes & Accessories,Women's Clothing,Jeans,Auction,13,246,10000095,1,Ebay
+2013-08-23,11848,0,Health & Beauty,Fragrances,Women,FP-GTC,14,109,10000096,1,Ebay
+2013-08-03,13836,0,Collectibles,Decorative Collectibles,Spoons,Auction,13,39.41,10000097,1,Ebay
+2013-05-17,13836,0,Collectibles,Decorative Collectibles,Spoons,Auction,14,16.26,10000098,1,Ebay
+2013-06-06,13987,0,Collectibles,Paper,Booklets,FP-GTC,13,112.56,10000099,1,Ebay
+2013-07-02,15687,0,ClothinShoes & Accessories,Men's Clothing,T-Shirts,Auction,14,184.21,10000100,1,Ebay
+2013-10-25,15687,0,ClothinShoes & Accessories,Men's Clothing,T-Shirts,Auction,11,27.48,10000001,1,Ebay
+2013-04-20,16145,3,Computers/Tablets & Networking,Computer Components & Parts,Other Components & Parts,FP-non GTC,12,26.45,10000002,1,Ebay
+2013-03-12,16145,0,Computers/Tablets & Networking,Computer Components & Parts,Other,FP-non GTC,13,415.73,10000003,1,Ebay
+2013-03-28,16509,0,Toys & Hobbies,Model Railroads & Trains,S Scale,ABIN,5,56.36,10000004,1,Ebay
+2013-10-29,16509,0,Toys & Hobbies,Model Railroads & Trains,S Scale,ABIN,5,2.44,10000005,1,Ebay
+2013-05-22,20485,0,Home & Garden,Furniture,Other,FP-GTC,14,269.76,10000006,1,Ebay
+2013-01-25,20485,101,CasArredamento e Bricolage,Cucina,Altro per cucina,FP-GTC,12,109,10000007,1,Ebay
+2013-06-12,20485,101,CasArredamento e Bricolage,Cucina,Altro per cucina,FP-GTC,12,101.79,10000008,1,Ebay
+2013-12-26,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,246,10000009,1,Ebay
+2013-12-26,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,189.23,10000010,1,Ebay
+2013-12-31,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,15.65,10000011,1,Ebay
+2013-10-04,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,28.23,10000012,1,Ebay
+2013-03-16,24541,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,College-NCAA,FP-GTC,5,16.26,10000013,1,Ebay
+2013-05-21,26262,0,Collectibles,Advertising,Food & Beverage,FP-GTC,5,122.78,10000014,1,Ebay
+2013-01-28,30059,3,Cameras & Photography,Lenses & Filters,Lens AdapterMounts & Tubes,FP-GTC,14,172.03,10000015,1,Ebay
+2013-04-26,31387,3,Jewellery & Watches,Watches,Wristwatches,Auction,14,42.99,10000016,1,Ebay
+2013-10-06,31387,3,Jewellery & Watches,Watches,Wristwatches,Auction,14,207.5,10000017,1,Ebay
+2013-11-06,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,5.91,10000018,1,Ebay
+2013-10-06,31519,3,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,39.41,10000019,1,Ebay
+2013-12-28,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,16.26,10000020,1,Ebay
+2013-11-06,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,16.26,10000021,1,Ebay
+2013-11-06,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,78.48,10000022,1,Ebay
+2013-12-28,31519,3,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,190.22,10000023,1,Ebay
+2013-06-11,35570,100,eBay Motors,Parts & Accessories,Motorcycle Parts,FP-GTC,12,2.44,10000024,1,Ebay
+2013-01-10,36250,0,Sporting Goods,Hunting,Decoys,Auction,5,7.12,10000025,1,Ebay
+2013-09-17,38238,0,Home & Garden,Home Decor,Other,FP-non GTC,14,36.7,10000026,1,Ebay
+2013-08-14,40059,3,Mobile Phones & Communication,Radio Communication Equipment,Parts & Accessories,FP-GTC,14,35.72,10000027,1,Ebay
+2013-08-09,40059,3,Mobile Phones & Communication,Radio Communication Equipment,Parts & Accessories,FP-GTC,14,3.49,10000028,1,Ebay
+2013-12-02,41940,0,Business & Industrial,Manufacturing & Metalworking,Metalworking Tooling,FP-GTC,13,223.63,10000029,1,Ebay
+2013-02-01,41940,0,Business & Industrial,Manufacturing & Metalworking,Metalworking Tooling,FP-GTC,13,265.56,10000030,1,Ebay
+2013-07-28,43479,0,Cameras & Photo,Film Photography,Other,FP-non GTC,13,62.02,10000031,1,Ebay
+2013-06-16,44079,0,Sporting Goods,Exercise & Fitness,GyWorkout & Yoga,FP-GTC,12,46.44,10000032,1,Ebay
+2013-08-23,45238,101,Abbigliamento e accessori,Donna: Accessori,SciarpFoulard e Scialli,Auction,14,132.33,10000033,1,Ebay
+2013-06-15,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,Auction,13,448.8,10000034,1,Ebay
+2013-06-15,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,FP-non GTC,14,207.5,10000035,1,Ebay
+2013-06-01,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,FP-non GTC,14,190.22,10000036,1,Ebay
+2013-08-10,46575,0,Business & Industrial,Light Equipment & Tools,Air Tools,FP-GTC,14,16.71,10000037,1,Ebay
+2013-03-22,50508,0,Cameras & Photo,Camera & Photo Accessories,LCD Hoods,FP-non GTC,13,4.13,10000038,1,Ebay
+2013-12-25,50508,0,Cameras & Photo,Camera & Photo Accessories,LCD Hoods,FP-non GTC,13,1.88,10000039,1,Ebay
+2013-07-22,50677,0,Jewelry & Watches,Fashion Jewelry,Pins & Brooches,FP-GTC,13,491.32,10000040,1,Ebay
+2013-04-13,50677,0,Jewelry & Watches,Fashion Jewelry,Pins & Brooches,FP-GTC,5,2.44,10000041,1,Ebay
+2013-04-16,51582,0,ClothinShoes & Accessories,Kids' ClothinShoes & Accs,Girls' Clothing (Sizes 4 & Up),Auction,14,56.36,10000042,1,Ebay
+2013-08-21,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-GTC,13,15.85,10000043,1,Ebay
+2013-04-22,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-non GTC,14,2.44,10000044,1,Ebay
+2013-08-29,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-GTC,14,7.12,10000045,1,Ebay
+2013-05-16,57784,0,ClothinShoes & Accessories,Baby & Toddler Clothing,Boys' Clothing (Newborn-5T),Auction,14,35.72,10000046,1,Ebay
+2013-08-23,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,11,9.26,10000047,1,Ebay
+2013-07-10,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,3.18,10000048,1,Ebay
+2013-08-10,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,638.72,10000049,1,Ebay
+2013-08-23,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,141.7,10000050,1,Ebay
+2013-04-18,57990,0,ClothinShoes & Accessories,Men's Clothing,Casual Shirts,ABIN,13,12.19,10000051,1,Ebay
+2013-07-10,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,132.33,10000052,1,Ebay
+2013-06-16,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,ABIN,5,5.48,10000053,1,Ebay
+2013-07-15,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,11,1.88,10000054,1,Ebay
+2013-11-06,60340,0,Entertainment Memorabilia,Movie Memorabilia,Pressbooks,FP-GTC,14,12.85,10000055,1,Ebay
+2013-12-27,60340,0,Entertainment Memorabilia,Movie Memorabilia,Pressbooks,FP-GTC,14,62.02,10000056,1,Ebay
+2013-07-29,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,15.85,10000057,1,Ebay
+2013-11-17,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,9.26,10000058,1,Ebay
+2013-07-27,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,16.71,10000059,1,Ebay
+2013-07-29,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,20.6,10000060,1,Ebay
+2013-01-09,63861,3,ClotheShoes & Accessories,Women's Clothing,Dresses,Auction,5,1.88,10000061,1,Ebay
+2013-06-11,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,5,141.7,10000062,1,Ebay
+2013-01-10,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,5,1.88,10000063,1,Ebay
+2013-09-16,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Others,11,112.56,10000064,1,Ebay
+2013-01-14,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Others,11,94.45,10000065,1,Ebay
+2013-05-17,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Auction,14,78.48,10000066,1,Ebay
+2013-06-05,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,13,5.48,10000067,1,Ebay
+2013-05-24,63864,3,ClotheShoes & Accessories,Women's Clothing,Skirts,Auction,14,28.23,10000068,1,Ebay
+2013-05-15,63889,0,ClothinShoes & Accessories,Women's Shoes,Mixed Items & Lots,Others,13,3.49,10000069,1,Ebay
+2013-03-25,67698,2,Business & Industrial,Retail & Services,Jewellery Packaging & Display,FP-GTC,11,15.65,10000070,1,Ebay
+2013-03-09,67698,0,Business & Industrial,Retail & Services,Jewelry Packaging & Display,FP-GTC,11,5.48,10000071,1,Ebay
+2013-12-05,67698,0,Business & Industrial,Retail & Services,Jewelry Packaging & Display,FP-GTC,11,246,10000072,1,Ebay
+2013-04-18,73506,0,Collectibles,Decorative Collectibles,Tea PotSets,FP-non GTC,13,122.78,10000073,1,Ebay
+2013-11-01,75665,0,Home & Garden,YarGarden & Outdoor Living,Gardening Supplies,FP-GTC,14,223.63,10000074,1,Ebay
+2013-05-03,75708,3,Toys & Games,Action Figures,TMovies & Video Games,ABIN,5,141.7,10000075,1,Ebay
+2013-04-21,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,21.14,10000076,1,Ebay
+2013-03-12,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,55.89,10000077,1,Ebay
+2013-05-19,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,51.23,10000078,1,Ebay
+2013-11-23,80135,0,Computers/Tablets & Networking,DriveStorage & Blank Media,Blank Media & Accessories,Auction,14,21.72,10000079,1,Ebay
+2013-10-19,95672,3,ClotheShoes & Accessories,Women's Shoes,Trainers,Auction,14,204.28,10000080,1,Ebay
+2013-05-18,95672,0,ClothinShoes & Accessories,Women's Shoes,Athletic,Others,11,21.14,10000081,1,Ebay
+2013-02-01,100847,0,Half Books,Half Books,Half Books,Others,5,204.28,10000082,1,Ebay
+2013-01-14,100847,0,Half Books,Half Books,Half Books,Others,5,122.78,10000083,1,Ebay
+2013-08-05,139973,3,Video Games & Consoles,Games,Games,ABIN,14,94.45,10000084,1,Ebay
+2013-05-19,139973,0,Video Games & Consoles,Video Games,Video Games,ABIN,11,86.58,10000085,1,Ebay
+2013-12-01,150047,3,Crafts,Jewellery Making,Findings,Auction,14,56.36,10000086,1,Ebay
+2013-12-02,150047,3,Crafts,Jewellery Making,Findings,Auction,14,290.72,10000087,1,Ebay
+2013-01-11,155226,0,ClothinShoes & Accessories,Women's Clothing,Sweats & Hoodies,FP-GTC,13,60.37,10000088,1,Ebay
+2013-05-27,155226,0,ClothinShoes & Accessories,Women's Clothing,Sweats & Hoodies,FP-GTC,13,112.56,10000089,1,Ebay
+2013-09-01,156356,0,Collectibles,Postcards,BuildingArchitecture,FP-GTC,13,265.56,10000090,1,Ebay
+2013-04-11,158798,0,Toys & Hobbies,Vintage & Antique Toys,Spinning Tops,FP-GTC,11,35.72,10000091,1,Ebay
+2013-05-05,165888,0,Jewelry & Watches,Vintage & Antique Jewelry,Costume,FP-non GTC,13,92.98,10000092,1,Ebay
+2013-11-21,170083,3,Computers/Tablets & Networking,Computer Components & Parts,Memory (RAM),Auction,11,28.23,10000093,1,Ebay
+2013-10-07,170083,3,Computers/Tablets & Networking,Computer Components & Parts,Memory (RAM),Auction,11,27.48,10000094,1,Ebay
+2013-07-12,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,9.26,10000095,1,Ebay
+2013-06-07,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,3.18,10000096,1,Ebay
+2013-05-22,175750,0,Home & Garden,Bedding,Blankets & Throws,FP-GTC,14,12.04,10000097,1,Ebay
+2013-11-28,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,13,20.6,10000098,1,Ebay
+2013-07-12,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,13,12.04,10000099,1,Ebay
+2013-06-07,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,4.13,10000100,1,Ebay
+2013-12-01,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,73.26,10000201,1,Ebay
+2012-10-12,48027,0,Coins & Paper Money,Paper Money: World,Asia,Auction,12,184.21,10000001,1,Ebay
+2012-08-20,164261,0,Jewelry & Watches,Fashion Jewelry,Earrings,Others,5,172.03,10000002,1,Ebay
+2012-09-18,82494,15,BookMagazines,NULL,Comic Books,Auction,14,66.6,10000003,1,Ebay
+2012-06-30,66767,15,Home & Garden,NULL,Dogs,Auction,12,92.98,10000004,1,Ebay
+2012-08-29,152801,0,Jewelry & Watches,NULL,Earrings,FP-GTC,5,132.33,10000005,1,Ebay
+2012-01-06,43398,0,Home & Garden,NULL,Cheese & Crackers,FP-GTC,13,7.12,10000006,1,Ebay
+2012-08-14,95173,0,Health & Beauty,Bath & Body,Bath Sets & Kits,Auction,14,204.28,10000007,1,Ebay
+2012-07-17,158666,15,ToyHobbies,Action Figures,Anime & Manga,Auction,13,35.72,10000008,1,Ebay
+2012-07-23,12688,0,eBay Premier,Books & Manuscripts,Books: Other,Auction,12,4.13,10000009,1,Ebay
+2012-07-27,103324,15,ClothinShoeAccessories,Women's Shoes,Mixed Items,FP-GTC,5,27.48,10000010,1,Ebay
+2012-07-29,108782,15,Vehicle Parts & Accessories,CaTruck Parts,Car Care & Cleaning,FP-GTC,14,9.26,10000011,1,Ebay
+2012-11-17,80287,0,Computers/Tablets & Networking,Software,Office & Business,Auction,12,3.18,10000012,1,Ebay
+2012-10-29,140746,100,eBay Motors,Parts & Accessories,Vintage Car & Truck Parts,Others,13,3.18,10000013,1,Ebay
+2012-03-28,87118,0,Sporting Goods,Outdoor Sports,Paintball,ABIN,14,377.94,10000014,1,Ebay
+2012-01-30,25147,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Baseball-MLB,Auction,12,146.33,10000015,1,Ebay
+2012-10-07,170302,15,Crafts,Embroidery,Design CDs,FP-GTC,5,51.23,10000016,1,Ebay
+2012-11-28,53064,0,Business & Industrial,Heavy Equipment,Antique & Vintage Farm Equip,FP-non GTC,13,72.65,10000017,1,Ebay
+2012-07-12,132939,0,Jewelry & Watches,Fashion Jewelry,Other,Auction,13,66.6,10000018,1,Ebay
+2012-06-07,113593,15,Phones,Mobile Phones,Mobile Phones,Auction,12,9.26,10000019,1,Ebay
+2012-12-01,34273,100,eBay Motors,Parts & Accessories,Motorcycle,Auction,14,583.44,10000020,1,Ebay
+2012-12-02,106340,15,Home & Garden,Gardening,Hand Tools,FP-GTC,14,638.72,10000021,1,Ebay
+2012-02-01,150265,15,Baby,Baby Clothing,Boys,FP-GTC,14,4.54,10000022,1,Ebay
+2012-01-14,24760,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Hockey-NHL,FP-GTC,12,319.79,10000023,1,Ebay
+2012-08-14,37831,0,Collectibles,Advertising,Merchandise & Memorabilia,Auction,12,20.35,10000024,1,Ebay
+2012-10-04,1120,3,Books,First Editions,Other,FP-non GTC,5,223.63,10000025,1,Ebay
+2012-12-26,43972,100,eBay Motors,Parts & Accessories,ATV Parts,FP-GTC,13,204.28,10000026,1,Ebay
+2012-12-31,166013,15,Computers,Computer Components & Parts,Video Capture & TV Tuner Cards,Auction,14,5.48,10000027,1,Ebay
+2012-03-22,15568,15,Baby,Baby Clothing,Unisex,Auction,14,27.48,10000028,1,Ebay
+2012-12-25,103178,15,ClothinShoeAccessories,Women's Bags,Women's Bags,FP-GTC,5,21.72,10000029,1,Ebay
+2012-12-05,2023,0,Sporting Goods,Team Sports,Basketball,ABIN,12,3.18,10000030,1,Ebay
+2012-05-27,94847,0,Consumer Electronics,Vehicle Electronics & GPS,Car Video,FP-GTC,11,491.32,10000031,1,Ebay
+2012-01-11,15868,0,Real Estate,Land,Land,Auction,14,448.8,10000032,1,Ebay
+2012-02-04,32876,0,Home & Garden,Home Improvement,Plumbing & Fixtures,Auction,13,415.73,10000033,1,Ebay
+2012-12-15,62179,0,ClothinShoes & Accessories,Women's Clothing,Athletic Apparel,Auction,13,377.94,10000034,1,Ebay
+2012-06-30,33038,15,Musical Instruments,Instruments,Guitars (Electric),FP-GTC,14,146.33,10000035,1,Ebay
+2012-12-16,156614,0,Toys & Hobbies,Diecast & Toy Vehicles,Cars: RacinNASCAR,FP-GTC,5,7.12,10000036,1,Ebay
+2012-10-17,106246,0,Health & Beauty,Hair Care & Styling,Shampoo & Conditioning,Auction,13,42.99,10000037,1,Ebay
+2012-10-08,20865,0,ClothinShoes & Accessories,Men's Clothing,Athletic Apparel,Auction,13,12.85,10000038,1,Ebay
+2012-04-26,15115,0,Video Games & Consoles,Video Games,Video Games,FP-GTC,13,55.89,10000039,1,Ebay
+2012-10-06,3838,0,Jewelry & Watches,Fashion Jewelry,Charms & Charm Bracelets,FP-GTC,14,73.26,10000040,1,Ebay
+2012-12-28,759,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,Auction,11,112.56,10000041,1,Ebay
+2012-11-06,61323,0,Consumer Electronics,TVideo & Home Audio,TVideo & Audio Accessories,FP-non GTC,11,3.49,10000042,1,Ebay
+2012-11-06,121153,0,Baby,Nursery Decor,Night Lights,Auction,13,184.21,10000043,1,Ebay
+2012-12-27,88750,0,Consumer Electronics,Vehicle Electronics & GPS,Radar & Laser Detectors,Auction,13,157.14,10000044,1,Ebay
+2012-07-15,161567,15,Computers,Laptop & Desktop Accessories,Laptop Batteries,FP-GTC,14,72.65,10000045,1,Ebay
+2012-08-23,113802,15,Lots More...,Metaphysical,Herbs,FP-GTC,14,51.23,10000046,1,Ebay
+2012-07-10,15808,15,ClothinShoeAccessories,Women's Clothing,Tops & Blouses,FP-non GTC,14,15.85,10000047,1,Ebay
+2012-08-10,174053,3,Vehicle Parts & Accessories,Car Parts,External & Body Parts,FP-GTC,13,7.12,10000048,1,Ebay
+2012-09-01,2635,0,Toys & Hobbies,Toy Soldiers,1970-Now,Auction,14,12.04,10000049,1,Ebay
+2012-09-19,1161,3,DVFilm & TV,Other Formats,Videos: NTSC  (US),Auction,13,73.26,10000050,1,Ebay
+2012-08-10,64076,0,Computers/Tablets & Networking,Enterprise NetworkinServers,Switches & Hubs,FP-non GTC,5,184.21,10000051,1,Ebay
+2012-08-21,33977,15,Crafts,Scrapbooking,Albums,FP-GTC,13,172.03,10000052,1,Ebay
+2012-08-29,31673,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,Racing-NASCAR,FP-GTC,14,122.78,10000053,1,Ebay
+2012-04-22,174106,3,Vehicle Parts & Accessories,Car Parts,Transmission & Drivetrain,Auction,14,92.98,10000054,1,Ebay
+2012-07-02,26249,0,Business & Industrial,Printing & Graphic Arts,Commercial Printing Presses,Auction,13,12.19,10000055,1,Ebay
+2012-01-28,159184,0,Sporting Goods,Winter Sports,Snowboarding,FP-GTC,5,15.65,10000056,1,Ebay
+2012-03-11,10058,3,Events Tickets,Other Tickets,Other Tickets,FP-GTC,11,101.79,10000057,1,Ebay
+2012-05-29,48904,0,ClothinShoes & Accessories,Vintage,Women's Vintage Clothing,ABIN,12,7.12,10000058,1,Ebay
+2012-05-31,145970,0,Toys & Hobbies,Models & Kits,Automotive,FP-non GTC,14,12.85,10000059,1,Ebay
+2012-10-25,963,0,ClothinShoes & Accessories,Vintage,Women's Vintage Shoes,FP-GTC,13,12.19,10000060,1,Ebay
+2012-11-23,118687,3,Health & Beauty,Fragrances,Women's Fragrances,FP-GTC,13,92.98,10000061,1,Ebay
+2012-09-28,20886,0,Toys & Hobbies,Diecast & Toy Vehicles,Cars: RacinNASCAR,FP-GTC,14,42.99,10000062,1,Ebay
+2012-09-28,148324,15,Phones,Mobile Accessories,CaseCoverSkins,Auction,13,1.88,10000063,1,Ebay
+2012-10-19,139255,15,Jewellery & Watches,Fine Jewellery,Earrings,Auction,14,21.14,10000064,1,Ebay
+2012-08-05,20213,0,Collectibles,Postcards,US StateCities & Towns,FP-GTC,5,21.14,10000065,1,Ebay
+2012-05-19,32996,15,Movies,Television Memorabilia,Clippings,Auction,13,132.33,10000066,1,Ebay
+2012-04-11,99985,0,Collectibles,Trading Cards,Sci-FFantasy,FP-GTC,14,120.87,10000067,1,Ebay
+2012-05-16,67703,3,Jewellery & Watches,Jewellery Boxes & Supplies,Jewellery Display,Auction,14,120.87,10000068,1,Ebay
+2012-05-24,65,0,Collectibles,Comics,Platinum Age (1897-1937),FP-non GTC,11,9.26,10000069,1,Ebay
+2012-01-10,130,0,Collectibles,Transportation,Railroadiana & Trains,FP-non GTC,14,16.26,10000070,1,Ebay
+2012-05-17,164,0,Computers/Tablets & Networking,Computer Components & Parts,CPUProcessors,FP-GTC,14,157.14,10000071,1,Ebay
+2012-02-03,216,0,Sports MeCards & Fan Shop,Cards,Hockey,FP-GTC,11,1.88,10000072,1,Ebay
+2012-05-05,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,12.04,10000073,1,Ebay
+2012-11-26,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,14,189.23,10000074,1,Ebay
+2012-08-30,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,73.26,10000075,1,Ebay
+2012-05-21,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,4.13,10000076,1,Ebay
+2012-06-06,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,290.72,10000077,1,Ebay
+2012-06-16,223,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-non GTC,5,265.56,10000078,1,Ebay
+2012-05-15,279,15,BookMagazines,Children's Books,Children's Books,FP-GTC,5,5.91,10000079,1,Ebay
+2012-12-05,314,0,ClothinShoes & Accessories,Women's Clothing,Other,Auction,5,319.79,10000080,1,Ebay
+2012-03-25,314,211,ClothinShoes & Accessories,Womens' Clothing,Other,Auction,5,246,10000081,1,Ebay
+2012-03-09,314,211,ClothinShoes & Accessories,Womens' Clothing,Other,Auction,5,20.35,10000082,1,Ebay
+2012-05-06,314,0,ClothinShoes & Accessories,Women's Clothing,Other,Auction,5,36.7,10000083,1,Ebay
+2012-06-15,533,0,Coins & Paper Money,Coins: World,Africa,Auction,13,101.79,10000084,1,Ebay
+2012-03-14,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,5,47.71,10000085,1,Ebay
+2012-05-20,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,5,3.49,10000086,1,Ebay
+2012-05-17,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,13,46.44,10000087,1,Ebay
+2012-03-11,1349,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,13,4.54,10000088,1,Ebay
+2012-01-10,1357,0,Collectibles,Decorative Collectibles,Decorative Collectible Brands,ABIN,14,3.18,10000089,1,Ebay
+2012-04-13,1504,0,Business & Industrial,Electrical & Test Equipment,Test Equipment,FP-GTC,14,86.58,10000090,1,Ebay
+2012-07-22,4943,0,Toys & Hobbies,Diecast & Toy Vehicles,CarTrucks & Vans,FP-GTC,13,12.85,10000091,1,Ebay
+2012-06-15,6762,0,Unknown,Unknown,Unknown,ABIN,13,16.26,10000092,1,Ebay
+2012-06-15,9426,3,Mobile Phones & Communication,Home Phones & Accessories,Phone Accessories,Auction,13,21.14,10000093,1,Ebay
+2012-06-01,10866,0,Collectibles,Animals,Farm & Countryside,FP-non GTC,14,20.6,10000094,1,Ebay
+2012-05-17,11554,0,ClothinShoes & Accessories,Women's Clothing,Jeans,Auction,13,246,10000095,1,Ebay
+2012-04-26,11848,0,Health & Beauty,Fragrances,Women,FP-GTC,14,109,10000096,1,Ebay
+2012-07-28,13836,0,Collectibles,Decorative Collectibles,Spoons,Auction,13,39.41,10000097,1,Ebay
+2012-03-12,13836,0,Collectibles,Decorative Collectibles,Spoons,Auction,14,16.26,10000098,1,Ebay
+2012-04-20,13987,0,Collectibles,Paper,Booklets,FP-GTC,13,112.56,10000099,1,Ebay
+2012-03-18,15687,0,ClothinShoes & Accessories,Men's Clothing,T-Shirts,Auction,14,184.21,10000100,1,Ebay
+2012-05-17,15687,0,ClothinShoes & Accessories,Men's Clothing,T-Shirts,Auction,11,27.48,10000001,1,Ebay
+2012-06-23,16145,3,Computers/Tablets & Networking,Computer Components & Parts,Other Components & Parts,FP-non GTC,12,26.45,10000002,1,Ebay
+2012-05-22,16145,0,Computers/Tablets & Networking,Computer Components & Parts,Other,FP-non GTC,13,415.73,10000003,1,Ebay
+2012-01-25,16509,0,Toys & Hobbies,Model Railroads & Trains,S Scale,ABIN,5,56.36,10000004,1,Ebay
+2012-06-12,16509,0,Toys & Hobbies,Model Railroads & Trains,S Scale,ABIN,5,2.44,10000005,1,Ebay
+2012-05-17,20485,0,Home & Garden,Furniture,Other,FP-GTC,14,269.76,10000006,1,Ebay
+2012-08-03,20485,101,CasArredamento e Bricolage,Cucina,Altro per cucina,FP-GTC,12,109,10000007,1,Ebay
+2012-05-17,20485,101,CasArredamento e Bricolage,Cucina,Altro per cucina,FP-GTC,12,101.79,10000008,1,Ebay
+2012-08-21,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,246,10000009,1,Ebay
+2012-08-21,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,189.23,10000010,1,Ebay
+2012-04-18,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,15.65,10000011,1,Ebay
+2012-06-16,23446,23,Mode & Accessoires,Chaussures de femme,Sandales & Sandalettes,Auction,14,28.23,10000012,1,Ebay
+2012-01-09,24541,0,Sports MeCards & Fan Shop,Fan Apparel & Souvenirs,College-NCAA,FP-GTC,5,16.26,10000013,1,Ebay
+2012-05-03,26262,0,Collectibles,Advertising,Food & Beverage,FP-GTC,5,122.78,10000014,1,Ebay
+2012-05-20,30059,3,Cameras & Photography,Lenses & Filters,Lens AdapterMounts & Tubes,FP-GTC,14,172.03,10000015,1,Ebay
+2012-03-12,31387,3,Jewellery & Watches,Watches,Wristwatches,Auction,14,42.99,10000016,1,Ebay
+2012-05-19,31387,3,Jewellery & Watches,Watches,Wristwatches,Auction,14,207.5,10000017,1,Ebay
+2012-04-21,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,5.91,10000018,1,Ebay
+2012-05-18,31519,3,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,39.41,10000019,1,Ebay
+2012-06-16,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,16.26,10000020,1,Ebay
+2012-06-11,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,16.26,10000021,1,Ebay
+2012-03-31,31519,0,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,78.48,10000022,1,Ebay
+2012-11-12,31519,3,Computers/Tablets & Networking,Laptop & Desktop Accessories,Laptop Cases & Bags,FP-GTC,14,190.22,10000023,1,Ebay
+2012-04-06,35570,100,eBay Motors,Parts & Accessories,Motorcycle Parts,FP-GTC,12,2.44,10000024,1,Ebay
+2012-11-01,36250,0,Sporting Goods,Hunting,Decoys,Auction,5,7.12,10000025,1,Ebay
+2012-02-06,38238,0,Home & Garden,Home Decor,Other,FP-non GTC,14,36.7,10000026,1,Ebay
+2012-06-16,40059,3,Mobile Phones & Communication,Radio Communication Equipment,Parts & Accessories,FP-GTC,14,35.72,10000027,1,Ebay
+2012-09-12,40059,3,Mobile Phones & Communication,Radio Communication Equipment,Parts & Accessories,FP-GTC,14,3.49,10000028,1,Ebay
+2012-04-16,41940,0,Business & Industrial,Manufacturing & Metalworking,Metalworking Tooling,FP-GTC,13,223.63,10000029,1,Ebay
+2012-11-01,41940,0,Business & Industrial,Manufacturing & Metalworking,Metalworking Tooling,FP-GTC,13,265.56,10000030,1,Ebay
+2012-06-14,43479,0,Cameras & Photo,Film Photography,Other,FP-non GTC,13,62.02,10000031,1,Ebay
+2012-11-12,44079,0,Sporting Goods,Exercise & Fitness,GyWorkout & Yoga,FP-GTC,12,46.44,10000032,1,Ebay
+2012-03-22,45238,101,Abbigliamento e accessori,Donna: Accessori,SciarpFoulard e Scialli,Auction,14,132.33,10000033,1,Ebay
+2012-05-22,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,Auction,13,448.8,10000034,1,Ebay
+2012-03-10,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,FP-non GTC,14,207.5,10000035,1,Ebay
+2012-05-17,45333,0,ClothinShoes & Accessories,Women's Shoes,Flats & Oxfords,FP-non GTC,14,190.22,10000036,1,Ebay
+2012-09-01,46575,0,Business & Industrial,Light Equipment & Tools,Air Tools,FP-GTC,14,16.71,10000037,1,Ebay
+2012-04-18,50508,0,Cameras & Photo,Camera & Photo Accessories,LCD Hoods,FP-non GTC,13,4.13,10000038,1,Ebay
+2012-06-05,50508,0,Cameras & Photo,Camera & Photo Accessories,LCD Hoods,FP-non GTC,13,1.88,10000039,1,Ebay
+2012-10-08,50677,0,Jewelry & Watches,Fashion Jewelry,Pins & Brooches,FP-GTC,13,491.32,10000040,1,Ebay
+2012-09-17,50677,0,Jewelry & Watches,Fashion Jewelry,Pins & Brooches,FP-GTC,5,2.44,10000041,1,Ebay
+2012-02-02,51582,0,ClothinShoes & Accessories,Kids' ClothinShoes & Accs,Girls' Clothing (Sizes 4 & Up),Auction,14,56.36,10000042,1,Ebay
+2012-08-23,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-GTC,13,15.85,10000043,1,Ebay
+2012-08-17,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-non GTC,14,2.44,10000044,1,Ebay
+2012-05-05,57013,0,Business & Industrial,MRO & Industrial Supply,Pumps & Plumbing,FP-GTC,14,7.12,10000045,1,Ebay
+2012-03-08,57784,0,ClothinShoes & Accessories,Baby & Toddler Clothing,Boys' Clothing (Newborn-5T),Auction,14,35.72,10000046,1,Ebay
+2012-03-16,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,11,9.26,10000047,1,Ebay
+2012-05-09,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,3.18,10000048,1,Ebay
+2012-03-25,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,638.72,10000049,1,Ebay
+2012-06-05,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,141.7,10000050,1,Ebay
+2012-01-10,57990,0,ClothinShoes & Accessories,Men's Clothing,Casual Shirts,ABIN,13,12.19,10000051,1,Ebay
+2012-06-11,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,14,132.33,10000052,1,Ebay
+2012-08-23,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,ABIN,5,5.48,10000053,1,Ebay
+2012-08-20,57990,3,ClotheShoes & Accessories,Men's Clothing,Casual Shirts & Tops,Auction,11,1.88,10000054,1,Ebay
+2012-08-09,60340,0,Entertainment Memorabilia,Movie Memorabilia,Pressbooks,FP-GTC,14,12.85,10000055,1,Ebay
+2012-06-30,60340,0,Entertainment Memorabilia,Movie Memorabilia,Pressbooks,FP-GTC,14,62.02,10000056,1,Ebay
+2012-06-03,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,15.85,10000057,1,Ebay
+2012-08-14,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,9.26,10000058,1,Ebay
+2012-08-14,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,16.71,10000059,1,Ebay
+2012-05-17,60606,3,Collectables,Badges/ Patches,Golly Badges,FP-GTC,12,20.6,10000060,1,Ebay
+2012-07-23,63861,3,ClotheShoes & Accessories,Women's Clothing,Dresses,Auction,5,1.88,10000061,1,Ebay
+2012-07-27,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,5,141.7,10000062,1,Ebay
+2012-07-29,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,5,1.88,10000063,1,Ebay
+2012-11-17,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Others,11,112.56,10000064,1,Ebay
+2012-10-29,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Others,11,94.45,10000065,1,Ebay
+2012-03-28,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,Auction,14,78.48,10000066,1,Ebay
+2012-01-30,63861,0,ClothinShoes & Accessories,Women's Clothing,Dresses,ABIN,13,5.48,10000067,1,Ebay
+2012-01-26,63864,3,ClotheShoes & Accessories,Women's Clothing,Skirts,Auction,14,28.23,10000068,1,Ebay
+2012-11-21,63889,0,ClothinShoes & Accessories,Women's Shoes,Mixed Items & Lots,Others,13,3.49,10000069,1,Ebay
+2012-07-12,67698,2,Business & Industrial,Retail & Services,Jewellery Packaging & Display,FP-GTC,11,15.65,10000070,1,Ebay
+2012-06-07,67698,0,Business & Industrial,Retail & Services,Jewelry Packaging & Display,FP-GTC,11,5.48,10000071,1,Ebay
+2012-12-01,67698,0,Business & Industrial,Retail & Services,Jewelry Packaging & Display,FP-GTC,11,246,10000072,1,Ebay
+2012-12-02,73506,0,Collectibles,Decorative Collectibles,Tea PotSets,FP-non GTC,13,122.78,10000073,1,Ebay
+2012-02-01,75665,0,Home & Garden,YarGarden & Outdoor Living,Gardening Supplies,FP-GTC,14,223.63,10000074,1,Ebay
+2012-01-14,75708,3,Toys & Games,Action Figures,TMovies & Video Games,ABIN,5,141.7,10000075,1,Ebay
+2012-09-16,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,21.14,10000076,1,Ebay
+2012-08-09,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,55.89,10000077,1,Ebay
+2012-12-26,80053,0,Computers/Tablets & Networking,MonitorProjectors & Accs,Monitors,FP-non GTC,11,51.23,10000078,1,Ebay
+2012-12-31,80135,0,Computers/Tablets & Networking,DriveStorage & Blank Media,Blank Media & Accessories,Auction,14,21.72,10000079,1,Ebay
+2012-03-22,95672,3,ClotheShoes & Accessories,Women's Shoes,Trainers,Auction,14,204.28,10000080,1,Ebay
+2012-12-25,95672,0,ClothinShoes & Accessories,Women's Shoes,Athletic,Others,11,21.14,10000081,1,Ebay
+2012-01-15,100847,0,Half Books,Half Books,Half Books,Others,5,204.28,10000082,1,Ebay
+2012-05-27,100847,0,Half Books,Half Books,Half Books,Others,5,122.78,10000083,1,Ebay
+2012-01-11,139973,3,Video Games & Consoles,Games,Games,ABIN,14,94.45,10000084,1,Ebay
+2012-02-04,139973,0,Video Games & Consoles,Video Games,Video Games,ABIN,11,86.58,10000085,1,Ebay
+2012-12-15,150047,3,Crafts,Jewellery Making,Findings,Auction,14,56.36,10000086,1,Ebay
+2012-06-30,150047,3,Crafts,Jewellery Making,Findings,Auction,14,290.72,10000087,1,Ebay
+2012-12-16,155226,0,ClothinShoes & Accessories,Women's Clothing,Sweats & Hoodies,FP-GTC,13,60.37,10000088,1,Ebay
+2012-11-11,155226,0,ClothinShoes & Accessories,Women's Clothing,Sweats & Hoodies,FP-GTC,13,112.56,10000089,1,Ebay
+2012-10-08,156356,0,Collectibles,Postcards,BuildingArchitecture,FP-GTC,13,265.56,10000090,1,Ebay
+2012-04-26,158798,0,Toys & Hobbies,Vintage & Antique Toys,Spinning Tops,FP-GTC,11,35.72,10000091,1,Ebay
+2012-10-06,165888,0,Jewelry & Watches,Vintage & Antique Jewelry,Costume,FP-non GTC,13,92.98,10000092,1,Ebay
+2012-12-28,170083,3,Computers/Tablets & Networking,Computer Components & Parts,Memory (RAM),Auction,11,28.23,10000093,1,Ebay
+2012-11-06,170083,3,Computers/Tablets & Networking,Computer Components & Parts,Memory (RAM),Auction,11,27.48,10000094,1,Ebay
+2012-11-06,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,9.26,10000095,1,Ebay
+2012-12-27,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,3.18,10000096,1,Ebay
+2012-01-01,175750,0,Home & Garden,Bedding,Blankets & Throws,FP-GTC,14,12.04,10000097,1,Ebay
+2012-08-23,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,13,20.6,10000098,1,Ebay
+2012-07-10,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,13,12.04,10000099,1,Ebay
+2012-08-10,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,4.13,10000100,1,Ebay
+2012-07-19,175750,3,HomFurniture & DIY,Bedding,Blankets,Auction,14,73.26,10000201,1,Ebay

http://git-wip-us.apache.org/repos/asf/kylin/blob/3032bbbe/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
index 4fbc2a5..5e7b130 100644
--- a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
@@ -180,7 +180,7 @@ public class ITKylinQueryTest extends KylinTestBase {
         execAndCompQuery("src/test/resources/query/sql_distinct_dim", null, true);
     }
 
-    //@Test
+    @Test
     public void testPreciselyDistinctCountQuery() throws Exception {
         if ("left".equalsIgnoreCase(joinType)) {
             execAndCompQuery("src/test/resources/query/sql_distinct_precisely", null, true);

http://git-wip-us.apache.org/repos/asf/kylin/blob/3032bbbe/kylin-it/src/test/resources/query/sql_distinct_precisely/query03.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_distinct_precisely/query03.sql b/kylin-it/src/test/resources/query/sql_distinct_precisely/query03.sql
index 3bf72f1..dbc2fac 100644
--- a/kylin-it/src/test/resources/query/sql_distinct_precisely/query03.sql
+++ b/kylin-it/src/test/resources/query/sql_distinct_precisely/query03.sql
@@ -18,6 +18,7 @@
 
 select test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV
  , count(1) as TRANS_CNT, count(distinct seller_id) as seller_count
+ , count(distinct site_name) as site_count
  from test_kylin_fact
  inner JOIN edw.test_cal_dt as test_cal_dt
  ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt

http://git-wip-us.apache.org/repos/asf/kylin/blob/3032bbbe/kylin-it/src/test/resources/query/sql_distinct_precisely/query04.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_distinct_precisely/query04.sql b/kylin-it/src/test/resources/query/sql_distinct_precisely/query04.sql
index b9fcff4..69006ce 100644
--- a/kylin-it/src/test/resources/query/sql_distinct_precisely/query04.sql
+++ b/kylin-it/src/test/resources/query/sql_distinct_precisely/query04.sql
@@ -18,6 +18,7 @@
 
 select test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV
  , count(1) as TRANS_CNT, count(distinct seller_id) as seller_count
+ , count(distinct site_name) as site_count
  from test_kylin_fact
  inner JOIN edw.test_cal_dt as test_cal_dt
  ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt

http://git-wip-us.apache.org/repos/asf/kylin/blob/3032bbbe/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
index 0a53787..ae870a0 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
@@ -364,9 +364,13 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
                     build());
 
         } catch (IOException ioe) {
-            logger.error(ioe.toString());
+            logger.error(ioe.toString(), ioe);
             IOException wrapped = new IOException("Error in coprocessor " + debugGitTag, ioe);
             ResponseConverter.setControllerException(controller, wrapped);
+        } catch (OutOfMemoryError oom) {
+            logger.error(oom.toString(), oom);
+            IOException wrapped = new IOException("OOM in coprocessor " + debugGitTag, oom);
+            ResponseConverter.setControllerException(controller, wrapped);
         } finally {
             for (RegionScanner innerScanner : regionScanners) {
                 IOUtils.closeQuietly(innerScanner);


[4/6] kylin git commit: KYLIN-1728 Introduce dictionary metadata

Posted by su...@apache.org.
KYLIN-1728 Introduce dictionary metadata


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

Branch: refs/heads/KYLIN-1705-B2
Commit: e305bb6b975d51b8ebee0bb63157e6273e0a55f2
Parents: d1a9bab
Author: sunyerui <su...@gmail.com>
Authored: Fri May 27 01:27:01 2016 +0800
Committer: sunyerui <su...@gmail.com>
Committed: Fri Jun 3 18:59:35 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/cube/CubeSegment.java |  3 +-
 .../org/apache/kylin/cube/model/CubeDesc.java   | 30 ++++++++++++++++++--
 .../cube/model/CubeJoinedFlatTableDesc.java     | 19 +++++++++++++
 3 files changed, 48 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/e305bb6b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
index 047064d..7d78633 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
@@ -294,7 +294,8 @@ public class CubeSegment implements Comparable<CubeSegment>, IRealizationSegment
     }
 
     public Dictionary<String> getDictionary(TblColRef col) {
-        return CubeManager.getInstance(this.getCubeInstance().getConfig()).getDictionary(this, col);
+        TblColRef reuseCol = getCubeDesc().getDictionaryReuseColumn(col);
+        return CubeManager.getInstance(this.getCubeInstance().getConfig()).getDictionary(this, reuseCol);
     }
 
     public CubeDimEncMap getDimensionEncodingMap() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/e305bb6b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index 9081494..984a7e5 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -918,6 +918,10 @@ public class CubeDesc extends RootPersistentEntity {
         if (dictionaries != null) {
             for (DictionaryDesc dictDesc : dictionaries) {
                 dictDesc.init(this);
+                allColumns.add(dictDesc.getColumnRef());
+                if (dictDesc.getResuseColumnRef() != null) {
+                    allColumns.add(dictDesc.getResuseColumnRef());
+                }
             }
         }
     }
@@ -1057,13 +1061,28 @@ public class CubeDesc extends RootPersistentEntity {
         // remove columns that reuse other's dictionary
         if (dictionaries != null) {
             for (DictionaryDesc dictDesc : dictionaries) {
-                if (dictDesc.getResuseColumnRef() != null)
+                if (dictDesc.getResuseColumnRef() != null) {
                     result.remove(dictDesc.getColumnRef());
+                    result.add(dictDesc.getResuseColumnRef());
+                }
             }
         }
         
         return result;
     }
+
+    /** A column may reuse dictionary of another column, find the dict column, return same col if there's no reuse column*/
+    public TblColRef getDictionaryReuseColumn(TblColRef col) {
+        if (dictionaries == null) {
+            return col;
+        }
+        for (DictionaryDesc dictDesc : dictionaries) {
+            if (dictDesc.getColumnRef().equals(col) && dictDesc.getResuseColumnRef() != null) {
+                return dictDesc.getResuseColumnRef();
+            }
+        }
+        return col;
+    }
     
     /** Get a column which can be used in distributing the source table */
     public TblColRef getDistributedByColumn() {
@@ -1080,8 +1099,13 @@ public class CubeDesc extends RootPersistentEntity {
             return null;
         
         for (DictionaryDesc desc : dictionaries) {
-            if (col.equals(desc.getColumnRef()) && desc.getBuilderClass() != null)
-                return desc.getBuilderClass();
+            if (desc.getBuilderClass() != null) {
+                if (col.equals(desc.getResuseColumnRef())) {
+                    return desc.getBuilderClass();
+                } else if (col.equals(desc.getColumnRef())) {
+                    return desc.getBuilderClass();
+                }
+            }
         }
         return null;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e305bb6b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
index db1942b..c5271c5 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
@@ -118,6 +118,25 @@ public class CubeJoinedFlatTableDesc implements IJoinedFlatTableDesc {
             }
         }
 
+        if (cubeDesc.getDictionaries() != null) {
+            for (DictionaryDesc dictDesc : cubeDesc.getDictionaries()) {
+                TblColRef c = dictDesc.getColumnRef();
+                if (contains(columnList, c) < 0) {
+                    columnIndexMap.put(colName(c.getCanonicalName()), columnIndex);
+                    columnList.add(new IntermediateColumnDesc(String.valueOf(columnIndex), c));
+                    columnIndex++;
+                }
+                if (dictDesc.getResuseColumnRef() != null) {
+                    c = dictDesc.getResuseColumnRef();
+                    if (contains(columnList, c) < 0) {
+                        columnIndexMap.put(colName(c.getCanonicalName()), columnIndex);
+                        columnList.add(new IntermediateColumnDesc(String.valueOf(columnIndex), c));
+                        columnIndex++;
+                    }
+                }
+            }
+        }
+
         columnCount = columnIndex;
     }
 


[6/6] kylin git commit: KYLIN-1379 More stable and functional precise count distinct implements after KYLIN-1186

Posted by su...@apache.org.
KYLIN-1379 More stable and functional precise count distinct implements after KYLIN-1186


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

Branch: refs/heads/KYLIN-1705-B2
Commit: 3032bbbe69f9e15d97e11d8ff0362ad034cdce4a
Parents: e305bb6
Author: sunyerui <su...@gmail.com>
Authored: Sat May 28 17:50:54 2016 +0800
Committer: sunyerui <su...@gmail.com>
Committed: Fri Jun 3 18:59:35 2016 +0800

----------------------------------------------------------------------
 .../validation/rule/DictionaryRuleTest.java     |   8 +-
 .../kylin/dict/AppendTrieDictionaryTest.java    |  18 +
 .../kylin/measure/bitmap/BitmapCounter.java     |   5 +
 .../kylin/measure/bitmap/BitmapMeasureType.java |  82 +-
 ...t_kylin_cube_without_slr_left_join_desc.json |  22 +-
 .../flatten_data_for_without_slr_left_join.csv  | 804 +++++++++----------
 .../apache/kylin/query/ITKylinQueryTest.java    |   2 +-
 .../query/sql_distinct_precisely/query03.sql    |   1 +
 .../query/sql_distinct_precisely/query04.sql    |   1 +
 .../coprocessor/endpoint/CubeVisitService.java  |   6 +-
 10 files changed, 508 insertions(+), 441 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/3032bbbe/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
index ba58d40..99ca7cf 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
@@ -71,16 +71,16 @@ public class DictionaryRuleTest extends LocalFileMetadataTestCase {
 
     @Test
     public void testBadDesc() throws IOException {
-        testBadDictionaryDesc("Column DEFAULT.TEST_KYLIN_FACT.SELLER_ID has inconsistent builders " +
+        testBadDictionaryDesc("Column EDW.TEST_SITES.SITE_NAME has inconsistent builders " +
                 "FakeBuilderClass and org.apache.kylin.dict.GlobalDictionaryBuilder",
-            DictionaryDesc.create("SELLER_ID", null, "FakeBuilderClass"));
+            DictionaryDesc.create("SITE_NAME", null, "FakeBuilderClass"));
     }
 
     @Test
     public void testBadDesc2() throws IOException {
-        testBadDictionaryDesc("Column DEFAULT.TEST_KYLIN_FACT.SELLER_ID has inconsistent builders " +
+        testBadDictionaryDesc("Column EDW.TEST_SITES.SITE_NAME has inconsistent builders " +
                         "FakeBuilderClass and org.apache.kylin.dict.GlobalDictionaryBuilder",
-                DictionaryDesc.create("lstg_site_id", "SELLER_ID", "FakeBuilderClass"));
+                DictionaryDesc.create("lstg_site_id", "SITE_NAME", "FakeBuilderClass"));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/kylin/blob/3032bbbe/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
index 10bbb77..e7603ec 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/AppendTrieDictionaryTest.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
 package org.apache.kylin.dict;
 
 import org.apache.hadoop.conf.Configuration;

http://git-wip-us.apache.org/repos/asf/kylin/blob/3032bbbe/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
index 6c90275..bbf812a 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
@@ -22,6 +22,7 @@ import org.roaringbitmap.buffer.MutableRoaringBitmap;
 
 import java.io.*;
 import java.nio.ByteBuffer;
+import java.util.Iterator;
 
 /**
  * Created by sunyerui on 15/12/1.
@@ -75,6 +76,10 @@ public class BitmapCounter implements Comparable<BitmapCounter> {
         return this.bitmap.getSizeInBytes();
     }
 
+    public Iterator<Integer> iterator() {
+        return bitmap.iterator();
+    }
+
     public void writeRegisters(ByteBuffer out) throws IOException {
         ByteArrayOutputStream bos = new ByteArrayOutputStream();
         DataOutputStream dos = new DataOutputStream(bos);

http://git-wip-us.apache.org/repos/asf/kylin/blob/3032bbbe/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
index 1b0edbf..3b0fe29 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
@@ -18,6 +18,8 @@
 
 package org.apache.kylin.measure.bitmap;
 
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.measure.MeasureIngester;
@@ -30,6 +32,7 @@ import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -76,11 +79,6 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
 
         if (DATATYPE_BITMAP.equals(functionDesc.getReturnDataType().getName()) == false)
             throw new IllegalArgumentException("BitmapMeasureType datatype is not " + DATATYPE_BITMAP + " but " + functionDesc.getReturnDataType().getName());
-
-        List<TblColRef> colRefs = functionDesc.getParameter().getColRefs();
-        if (colRefs.size() != 1 && colRefs.size() != 2) {
-            throw new IllegalArgumentException("Bitmap measure need 1 or 2 parameters, but has " + colRefs.size());
-        }
     }
 
     @Override
@@ -95,25 +93,48 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
 
             @Override
             public BitmapCounter valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
-                List<TblColRef> literalCols = measureDesc.getFunction().getParameter().getColRefs();
-                TblColRef literalCol = null;
-                if (literalCols.size() == 1) {
-                    literalCol = literalCols.get(0);
-                } else if (literalCols.size() == 2) {
-                    literalCol = literalCols.get(1);
-                } else {
-                    throw new IllegalArgumentException("Bitmap measure need 1 or 2 parameters");
-                }
-                Dictionary<String> dictionary = dictionaryMap.get(literalCol);
                 BitmapCounter bitmap = current;
                 bitmap.clear();
-                // bitmap measure may have two values due to two parameters, only the first value should be ingested
-                if (values != null && values.length > 0 && values[0] != null) {
-                    int id = dictionary.getIdFromValue(values[0]);
-                    bitmap.add(id);
+                if (needDictionaryColumn(measureDesc.getFunction())) {
+                    TblColRef literalCol = measureDesc.getFunction().getParameter().getColRefs().get(0);
+                    Dictionary<String> dictionary = dictionaryMap.get(literalCol);
+                    if (values != null && values.length > 0 && values[0] != null) {
+                        int id = dictionary.getIdFromValue(values[0]);
+                        bitmap.add(id);
+                    }
+                } else {
+                    for (String value : values) {
+                        bitmap.add(value);
+                    }
                 }
                 return bitmap;
             }
+
+            @Override
+            public BitmapCounter reEncodeDictionary(BitmapCounter value, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> oldDicts, Map<TblColRef, Dictionary<String>> newDicts) {
+                if (!needDictionaryColumn(measureDesc.getFunction())) {
+                    return value;
+                }
+                TblColRef colRef = measureDesc.getFunction().getParameter().getColRefs().get(0);
+                Dictionary<String> sourceDict = oldDicts.get(colRef);
+                Dictionary<String> mergedDict = newDicts.get(colRef);
+
+                BitmapCounter retValue = new BitmapCounter();
+                byte[] literal = new byte[sourceDict.getSizeOfValue()];
+                Iterator<Integer> iterator = value.iterator();
+                while (iterator.hasNext()) {
+                    int id = iterator.next();
+                    int newId;
+                    int size = sourceDict.getValueBytesFromId(id, literal, 0);
+                    if (size < 0) {
+                        newId = mergedDict.nullId();
+                    } else {
+                        newId = mergedDict.getIdFromValueBytes(literal, 0, size);
+                    }
+                    retValue.add(newId);
+                }
+                return retValue;
+            }
         };
     }
 
@@ -122,21 +143,12 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
         return new BitmapAggregator();
     }
 
-    /**
-     * generate dict with first col by default, and with second col if specified
-     *
-     * Typical case: we have col uuid, and another col flag_uuid (if flag==1, uuid, null),
-     * the metrics count(distinct uuid) and count(distinct flag_uuid) should both generate dict with uuid, instead of uuid and flag_uuid
-     */
     @Override
     public List<TblColRef> getColumnsNeedDictionary(FunctionDesc functionDesc) {
-        List<TblColRef> literalCols = functionDesc.getParameter().getColRefs();
-        if (literalCols.size() == 1) {
-            return Collections.singletonList(literalCols.get(0));
-        } else if (literalCols.size() == 2) {
-            return Collections.singletonList(literalCols.get(1));
+        if (needDictionaryColumn(functionDesc)) {
+            return Collections.singletonList(functionDesc.getParameter().getColRefs().get(0));
         } else {
-            throw new IllegalArgumentException("Bitmap measure need 1 or 2 parameters");
+            return Collections.emptyList();
         }
     }
 
@@ -150,4 +162,12 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
         return BitmapDistinctCountAggFunc.class;
     }
 
+    // In order to keep compatibility with old version, tinyint/smallint/int column use value directly, without dictionary
+    private boolean needDictionaryColumn(FunctionDesc functionDesc) {
+        DataType dataType = functionDesc.getParameter().getColRefs().get(0).getType();
+        if (dataType.isIntegerFamily() && !dataType.isBigInt()) {
+            return false;
+        }
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/3032bbbe/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
index 3361a3d..a501bd1 100644
--- a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
+++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
@@ -112,7 +112,19 @@
         "value" : "SELLER_ID",
         "next_parameter" : null
       },
-      "returntype" : "hllc(10)"
+      "returntype" : "bitmap"
+    },
+    "dependent_measure_ref" : null
+  }, {
+    "name" : "SITE_NAME_BITMAP",
+    "function" : {
+      "expression" : "COUNT_DISTINCT",
+      "parameter" : {
+        "type" : "column",
+        "value" : "SITE_NAME",
+        "next_parameter" : null
+      },
+      "returntype" : "bitmap"
     },
     "dependent_measure_ref" : null
   }, {
@@ -196,6 +208,12 @@
     },
     "dependent_measure_ref" : null
   } ],
+  "dictionaries" : [
+    {
+      "column" : "SITE_NAME",
+      "builder": "org.apache.kylin.dict.GlobalDictionaryBuilder"
+    }
+  ],
   "rowkey" : {
     "rowkey_columns" : [ {
       "column" : "cal_dt",
@@ -238,7 +256,7 @@
       "name" : "f2",
       "columns" : [ {
         "qualifier" : "m",
-        "measure_refs" : [ "seller_cnt_bitmap", "seller_format_cnt"]
+        "measure_refs" : [ "seller_cnt_bitmap", "site_name_bitmap", "seller_format_cnt"]
       } ]
     }, {
       "name" : "f3",