You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2016/09/03 06:37:55 UTC

[48/50] [abbrv] kylin git commit: KYLIN-1834 dict offset can be upto 5 bytes, trie dict is 2 GB at most

KYLIN-1834 dict offset can be upto 5 bytes, trie dict is 2 GB at most


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 67bcec20fb6fee502a5267dfa583bd07ca5edc8b
Parents: 7dae977
Author: Yang Li <li...@apache.org>
Authored: Sat Sep 3 13:13:00 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sat Sep 3 13:13:00 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/util/BytesUtil.java |   2 +-
 .../org/apache/kylin/dict/TrieDictionary.java   |  30 +++---
 .../kylin/dict/TrieDictionaryBuilder.java       | 105 +++++++------------
 .../apache/kylin/dict/TrieDictionaryTest.java   |  74 ++++++++++---
 4 files changed, 121 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/67bcec20/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
index bda5c73..759ddbd 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
@@ -98,7 +98,7 @@ public class BytesUtil {
     /**
      * No. bytes needed to store a value as big as the given
      */
-    public static int sizeForValue(int maxValue) {
+    public static int sizeForValue(long maxValue) {
         int size = 0;
         while (maxValue > 0) {
             size++;

http://git-wip-us.apache.org/repos/asf/kylin/blob/67bcec20/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
index 03dc76a..aea9551 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
@@ -58,8 +58,8 @@ import com.google.common.base.Preconditions;
 public class TrieDictionary<T> extends Dictionary<T> {
     private static final long serialVersionUID = 1L;
 
-    public static final byte[] HEAD_MAGIC = new byte[] { 0x54, 0x72, 0x69, 0x65, 0x44, 0x69, 0x63, 0x74 }; // "TrieDict"
-    public static final int HEAD_SIZE_I = HEAD_MAGIC.length;
+    public static final byte[] MAGIC = new byte[] { 0x54, 0x72, 0x69, 0x65, 0x44, 0x69, 0x63, 0x74 }; // "TrieDict"
+    public static final int MAGIC_SIZE_I = MAGIC.length;
 
     public static final int BIT_IS_LAST_CHILD = 0x80;
     public static final int BIT_IS_END_OF_VALUE = 0x40;
@@ -80,7 +80,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
 
     transient private int nValues;
     transient private int sizeOfId;
-    transient private int childOffsetMask;
+    transient private long childOffsetMask;
     transient private int firstByteOffset;
 
     transient private boolean enableValueCache = true;
@@ -99,12 +99,12 @@ public class TrieDictionary<T> extends Dictionary<T> {
 
     private void init(byte[] trieBytes) {
         this.trieBytes = trieBytes;
-        if (BytesUtil.compareBytes(HEAD_MAGIC, 0, trieBytes, 0, HEAD_MAGIC.length) != 0)
+        if (BytesUtil.compareBytes(MAGIC, 0, trieBytes, 0, 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));
+                    new ByteArrayInputStream(trieBytes, MAGIC_SIZE_I, trieBytes.length - MAGIC_SIZE_I));
             this.headSize = headIn.readShort();
             this.bodyLen = headIn.readInt();
             this.sizeChildOffset = headIn.read();
@@ -118,7 +118,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
 
             this.nValues = BytesUtil.readUnsigned(trieBytes, headSize + sizeChildOffset, sizeNoValuesBeneath);
             this.sizeOfId = BytesUtil.sizeForValue(baseId + nValues + 1); // note baseId could raise 1 byte in ID space, +1 to reserve all 0xFF for NULL case
-            this.childOffsetMask = ~((BIT_IS_LAST_CHILD | BIT_IS_END_OF_VALUE) << ((sizeChildOffset - 1) * 8));
+            this.childOffsetMask = ~((long) (BIT_IS_LAST_CHILD | BIT_IS_END_OF_VALUE) << ((sizeChildOffset - 1) * 8));
             this.firstByteOffset = sizeChildOffset + sizeNoValuesBeneath + 1; // the offset from begin of node to its first value byte
         } catch (Exception e) {
             if (e instanceof RuntimeException)
@@ -229,7 +229,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
                 seq++;
 
             // find a child to continue
-            int c = headSize + (BytesUtil.readUnsigned(trieBytes, n, sizeChildOffset) & childOffsetMask);
+            int c = getChildOffset(n);
             if (c == headSize) // has no children
                 return roundSeqNo(roundingFlag, seq - 1, -1, seq); // input only partially matched
             byte inpByte = inp[o];
@@ -253,6 +253,12 @@ public class TrieDictionary<T> extends Dictionary<T> {
         }
     }
 
+    private int getChildOffset(int n) {
+        long offset = headSize + (BytesUtil.readLong(trieBytes, n, sizeChildOffset) & childOffsetMask);
+        assert offset < trieBytes.length;
+        return (int) offset;
+    }
+
     private int roundSeqNo(int roundingFlag, int i, int j, int k) {
         if (roundingFlag == 0)
             return j;
@@ -338,7 +344,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
             }
 
             // find a child to continue
-            int c = headSize + (BytesUtil.readUnsigned(trieBytes, n, sizeChildOffset) & childOffsetMask);
+            int c = getChildOffset(n);
             if (c == headSize) // has no children
                 return -1; // no child? corrupted dictionary!
             int nValuesBeneath;
@@ -401,7 +407,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
         }
 
         // find a child to continue
-        int c = headSize + (BytesUtil.readUnsigned(trieBytes, n, sizeChildOffset) & childOffsetMask);
+        int c = getChildOffset(n);
         if (c == headSize) // has no children 
             return;
 
@@ -446,14 +452,14 @@ public class TrieDictionary<T> extends Dictionary<T> {
 
     @Override
     public void readFields(DataInput in) throws IOException {
-        byte[] headPartial = new byte[HEAD_MAGIC.length + Short.SIZE + Integer.SIZE];
+        byte[] headPartial = new byte[MAGIC.length + Short.SIZE + Integer.SIZE];
         in.readFully(headPartial);
 
-        if (BytesUtil.compareBytes(HEAD_MAGIC, 0, headPartial, 0, HEAD_MAGIC.length) != 0)
+        if (BytesUtil.compareBytes(MAGIC, 0, headPartial, 0, 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));
+                new ByteArrayInputStream(headPartial, MAGIC_SIZE_I, headPartial.length - MAGIC_SIZE_I));
         int headSize = headIn.readShort();
         int bodyLen = headIn.readInt();
         headIn.close();

http://git-wip-us.apache.org/repos/asf/kylin/blob/67bcec20/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
index 02da741..1271483 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
@@ -38,6 +38,8 @@ import org.apache.kylin.common.util.BytesUtil;
  * @author yangli9
  */
 public class TrieDictionaryBuilder<T> {
+    
+    private static final int _2GB = 2000000000;
 
     public static class Node {
         public byte[] part;
@@ -112,8 +114,7 @@ public class TrieDictionaryBuilder<T> {
             return;
         }
 
-        // if partially matched the current, split the current node, add the new
-        // value, make a 3-way
+        // if partially matched the current, split the current node, add the new value, make a 3-way
         if (i < n) {
             Node c1 = new Node(BytesUtil.subarray(node.part, i, n), node.isEndOfValue, node.children);
             Node c2 = new Node(BytesUtil.subarray(value, j, nn), true);
@@ -128,8 +129,7 @@ public class TrieDictionaryBuilder<T> {
             return;
         }
 
-        // out matched the current, binary search the next byte for a child node
-        // to continue
+        // 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;
@@ -188,28 +188,21 @@ public class TrieDictionaryBuilder<T> {
         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 long mbpn_nChildLookups; // number of child lookups during lookup every value once
+        public long 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_sizeNoValueBeneath; // size of field noValuesBeneath,
-                                            // depends on cardinality
-        public int mbpn_sizeChildOffset; // size of field childOffset, points to
-                                         // first child in flattened array
-        public int mbpn_footprint; // MBPN footprint in bytes
+        public int mbpn_sizeNoValueBeneath; // size of field noValuesBeneath, depends on cardinality
+        public int mbpn_sizeChildOffset; // size of field childOffset, points to first child in flattened array
+        public long mbpn_footprint; // MBPN footprint in bytes
 
         // stats for one-byte-per-node as well, so there's comparison
         public int obpn_sizeValue; // size of value per node, always 1
-        public int obpn_sizeNoValuesBeneath; // size of field noValuesBeneath,
-                                             // depends on cardinality
-        public int obpn_sizeChildCount; // size of field childCount, enables
-                                        // binary search among children
-        public int obpn_sizeChildOffset; // size of field childOffset, points to
-                                         // first child in flattened array
+        public int obpn_sizeNoValuesBeneath; // size of field noValuesBeneath, depends on cardinality
+        public int obpn_sizeChildCount; // size of field childCount, enables binary search among children
+        public int obpn_sizeChildOffset; // size of field childOffset, points to first child in flattened array
         public int obpn_nNodes; // no. nodes in OBPN trie
-        public int obpn_footprint; // OBPN footprint in bytes
+        public long obpn_footprint; // OBPN footprint in bytes
 
         public void print() {
             PrintStream out = System.out;
@@ -289,23 +282,12 @@ public class TrieDictionaryBuilder<T> {
         s.obpn_sizeValue = 1;
         s.obpn_sizeNoValuesBeneath = BytesUtil.sizeForValue(s.nValues);
         s.obpn_sizeChildCount = 1;
-        s.obpn_sizeChildOffset = 4; // MSB used as isEndOfValue flag
-        s.obpn_nNodes = s.nValueBytesCompressed; // no. nodes is the total
-                                                 // number of compressed
-                                                 // bytes in OBPN
-        s.obpn_footprint = s.obpn_nNodes * (s.obpn_sizeValue + s.obpn_sizeNoValuesBeneath + s.obpn_sizeChildCount + s.obpn_sizeChildOffset);
+        s.obpn_sizeChildOffset = 5; // MSB used as isEndOfValue flag
+        s.obpn_nNodes = s.nValueBytesCompressed; // no. nodes is the total number of compressed bytes in OBPN
+        s.obpn_footprint = s.obpn_nNodes * (long) (s.obpn_sizeValue + s.obpn_sizeNoValuesBeneath + s.obpn_sizeChildCount + s.obpn_sizeChildOffset);
         while (true) { // minimize the offset size to match the footprint
-            int t = s.obpn_nNodes * (s.obpn_sizeValue + s.obpn_sizeNoValuesBeneath + s.obpn_sizeChildCount + s.obpn_sizeChildOffset - 1);
-            if (BytesUtil.sizeForValue(t * 2) <= s.obpn_sizeChildOffset - 1) { // *2
-                                                                                   // because
-                                                                               // MSB
-                                                                               // of
-                                                                               // offset
-                                                                               // is
-                                                                               // used
-                                                                               // for
-                                                                               // isEndOfValue
-                                                                               // flag
+            long t = s.obpn_nNodes * (long) (s.obpn_sizeValue + s.obpn_sizeNoValuesBeneath + s.obpn_sizeChildCount + s.obpn_sizeChildOffset - 1);
+            if (BytesUtil.sizeForValue(t * 2) <= s.obpn_sizeChildOffset - 1) { // *2 because MSB of offset is used for isEndOfValue flag
                 s.obpn_sizeChildOffset--;
                 s.obpn_footprint = t;
             } else
@@ -316,23 +298,11 @@ public class TrieDictionaryBuilder<T> {
         s.mbpn_sizeValueTotal = s.nValueBytesCompressed;
         s.mbpn_sizeNoValueBytes = 1;
         s.mbpn_sizeNoValueBeneath = BytesUtil.sizeForValue(s.nValues);
-        s.mbpn_sizeChildOffset = 4;
-        s.mbpn_footprint = s.mbpn_sizeValueTotal + s.mbpn_nNodes * (s.mbpn_sizeNoValueBytes + s.mbpn_sizeNoValueBeneath + s.mbpn_sizeChildOffset);
+        s.mbpn_sizeChildOffset = 5;
+        s.mbpn_footprint = s.mbpn_sizeValueTotal + s.mbpn_nNodes * (long) (s.mbpn_sizeNoValueBytes + s.mbpn_sizeNoValueBeneath + 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_sizeNoValueBeneath + s.mbpn_sizeChildOffset - 1);
-            if (BytesUtil.sizeForValue(t * 4) <= s.mbpn_sizeChildOffset - 1) { // *4
-                                                                                   // because
-                                                                               // 2
-                                                                               // MSB
-                                                                               // of
-                                                                               // offset
-                                                                               // is
-                                                                               // used
-                                                                               // for
-                                                                               // isEndOfValue
-                                                                               // &
-                                                                               // isEndChild
-                                                                               // flag
+            long t = s.mbpn_sizeValueTotal + s.mbpn_nNodes * (long) (s.mbpn_sizeNoValueBytes + s.mbpn_sizeNoValueBeneath + s.mbpn_sizeChildOffset - 1);
+            if (BytesUtil.sizeForValue(t * 4) <= s.mbpn_sizeChildOffset - 1) { // *4 because 2 MSB of offset is used for isEndOfValue & isEndChild flag
                 s.mbpn_sizeChildOffset--;
                 s.mbpn_footprint = t;
             } else
@@ -415,8 +385,7 @@ public class TrieDictionaryBuilder<T> {
             }
         }
 
-        completeParts.append(node.part);// by here the node.children may have
-                                        // been changed
+        completeParts.append(node.part); // by here the node.children may have been changed
         for (Node child : node.children) {
             checkOverflowParts(child);
         }
@@ -427,11 +396,13 @@ public class TrieDictionaryBuilder<T> {
      * Flatten the trie into a byte array for a minimized memory footprint.
      * Lookup remains fast. Cost is inflexibility to modify (becomes immutable).
      * 
-     * Flattened node structure is HEAD + NODEs, for each node: - o byte, offset
-     * to child node, o = stats.mbpn_sizeChildOffset - 1 bit, isLastChild flag,
-     * the 1st MSB of o - 1 bit, isEndOfValue flag, the 2nd MSB of o - c byte,
-     * number of values beneath, c = stats.mbpn_sizeNoValueBeneath - 1 byte,
-     * number of value bytes - n byte, value bytes
+     * Flattened node structure is HEAD + NODEs, for each node:
+     * - o byte, offset to child node, o = stats.mbpn_sizeChildOffset
+     *    - 1 bit, isLastChild flag, the 1st MSB of o
+     *    - 1 bit, isEndOfValue flag, the 2nd MSB of o
+     * - c byte, number of values beneath, c = stats.mbpn_sizeNoValueBeneath
+     * - 1 byte, number of value bytes
+     * - n byte, value bytes
      */
     public TrieDictionary<T> build(int baseId) {
         byte[] trieBytes = buildTrieBytes(baseId);
@@ -445,15 +416,18 @@ public class TrieDictionaryBuilder<T> {
         Stats stats = stats();
         int sizeNoValuesBeneath = stats.mbpn_sizeNoValueBeneath;
         int sizeChildOffset = stats.mbpn_sizeChildOffset;
+        
+        if (stats.mbpn_footprint > _2GB)
+            throw new RuntimeException("Too big dictionary, dictionary cannot be bigger than 2GB");
 
         // write head
         byte[] head;
         try {
             ByteArrayOutputStream byteBuf = new ByteArrayOutputStream();
             DataOutputStream headOut = new DataOutputStream(byteBuf);
-            headOut.write(TrieDictionary.HEAD_MAGIC);
+            headOut.write(TrieDictionary.MAGIC);
             headOut.writeShort(0); // head size, will back fill
-            headOut.writeInt(stats.mbpn_footprint); // body size
+            headOut.writeInt((int) stats.mbpn_footprint); // body size
             headOut.write(sizeChildOffset);
             headOut.write(sizeNoValuesBeneath);
             headOut.writeShort(baseId);
@@ -461,13 +435,12 @@ public class TrieDictionaryBuilder<T> {
             headOut.writeUTF(bytesConverter == null ? "" : bytesConverter.getClass().getName());
             headOut.close();
             head = byteBuf.toByteArray();
-            BytesUtil.writeUnsigned(head.length, head, TrieDictionary.HEAD_SIZE_I, 2);
+            BytesUtil.writeUnsigned(head.length, head, TrieDictionary.MAGIC_SIZE_I, 2);
         } catch (IOException e) {
-            throw new RuntimeException(e); // shall not happen, as we are
-                                           // writing in memory
+            throw new RuntimeException(e); // shall not happen, as we are writing in memory
         }
 
-        byte[] trieBytes = new byte[stats.mbpn_footprint + head.length];
+        byte[] trieBytes = new byte[(int) stats.mbpn_footprint + head.length];
         System.arraycopy(head, 0, trieBytes, 0, head.length);
 
         LinkedList<Node> open = new LinkedList<Node>();
@@ -506,6 +479,8 @@ public class TrieDictionaryBuilder<T> {
 
     private int build_writeNode(Node n, int offset, boolean isLastChild, int sizeNoValuesBeneath, int sizeChildOffset, byte[] trieBytes) {
         int o = offset;
+        if (o > _2GB)
+            throw new IllegalStateException();
 
         // childOffset
         if (isLastChild)

http://git-wip-us.apache.org/repos/asf/kylin/blob/67bcec20/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryTest.java
index 90283b8..a87d7cb 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryTest.java
@@ -32,10 +32,10 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.io.UnsupportedEncodingException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.NoSuchElementException;
 import java.util.Random;
 import java.util.TreeSet;
 
@@ -44,12 +44,61 @@ import org.junit.Test;
 public class TrieDictionaryTest {
 
     public static void main(String[] args) throws Exception {
-        InputStream is = new FileInputStream("src/test/resources/dict/dw_category_grouping_names.dat");
-        // InputStream is =
-        // Util.getPackageResourceAsStream(TrieDictionaryTest.class,
-        // "eng_com.dic");
-        ArrayList<String> str = loadStrings(is);
-        benchmarkStringDictionary(str);
+        int count = (int) (Integer.MAX_VALUE * 0.8 / 64);
+        benchmarkStringDictionary(new RandomStrings(count));
+    }
+    
+    private static class RandomStrings implements Iterable<String> {
+        final private int size;
+
+        public RandomStrings(int size) {
+            this.size = size;
+            System.out.println("size = " + size);
+        }
+        
+        @Override
+        public Iterator<String> iterator() {
+            return new Iterator<String>() {
+                Random rand = new Random(1000);
+                int i = 0;
+
+                @Override
+                public boolean hasNext() {
+                    return i < size;
+                }
+
+                @Override
+                public String next() {
+                    if (hasNext() == false)
+                        throw new NoSuchElementException();
+                    
+                    i++;
+                    if (i % 1000000 == 0)
+                        System.out.println(i);
+                    
+                    return nextString();
+                }
+
+                private String nextString() {
+                    StringBuffer buf = new StringBuffer();
+                    for (int i = 0; i < 64; i++) {
+                        int v = rand.nextInt(16);
+                        char c;
+                        if (v >= 0 && v <= 9)
+                            c = (char) ('0' + v);
+                        else
+                            c = (char) ('a' + v - 10);
+                        buf.append(c);
+                    }
+                    return buf.toString();
+                }
+
+                @Override
+                public void remove() {
+                    throw new UnsupportedOperationException();
+                }
+            };
+        }
     }
 
     @Test
@@ -172,11 +221,11 @@ public class TrieDictionaryTest {
         testStringDictionary(str, null);
     }
 
-    private static void benchmarkStringDictionary(ArrayList<String> str) throws UnsupportedEncodingException {
+    private static void benchmarkStringDictionary(Iterable<String> str) throws IOException {
         TrieDictionaryBuilder<String> b = newDictBuilder(str);
         b.stats().print();
         TrieDictionary<String> dict = b.build(0);
-
+        
         TreeSet<String> set = new TreeSet<String>();
         for (String s : str) {
             set.add(s);
@@ -205,13 +254,14 @@ public class TrieDictionaryTest {
         // following jvm options may help
         // -XX:CompileThreshold=1500
         // -XX:+PrintCompilation
+        System.out.println("Benchmark awaitig...");
         benchmark("Warm up", dict, set, map, strArray, array);
         benchmark("Benchmark", dict, set, map, strArray, array);
     }
 
     private static int benchmark(String msg, TrieDictionary<String> dict, TreeSet<String> set, HashMap<String, Integer> map, String[] strArray, byte[][] array) {
         int n = set.size();
-        int times = 10 * 1000 * 1000 / n; // run 10 million lookups
+        int times = Math.max(10 * 1000 * 1000 / n, 1); // run 10 million lookups
         int keep = 0; // make sure JIT don't OPT OUT function calls under test
         byte[] valueBytes = new byte[dict.getSizeOfValue()];
         long start;
@@ -259,7 +309,7 @@ public class TrieDictionaryTest {
         }
         long timeIdToValueByDict = System.currentTimeMillis() - start;
         System.out.println(timeIdToValueByDict);
-
+        
         return keep;
     }
 
@@ -322,7 +372,7 @@ public class TrieDictionaryTest {
         }
     }
 
-    private static TrieDictionaryBuilder<String> newDictBuilder(ArrayList<String> str) {
+    private static TrieDictionaryBuilder<String> newDictBuilder(Iterable<String> str) {
         TrieDictionaryBuilder<String> b = new TrieDictionaryBuilder<String>(new StringBytesConverter());
         for (String s : str)
             b.addValue(s);