You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by gv...@apache.org on 2016/08/23 14:02:51 UTC

[1/2] incubator-carbondata git commit: Perform equal distribution of dictionary values among the sublists of a list whenever a dictionary file is loaded into memory

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master 62c0b05e6 -> 2d4609cdf


Perform equal distribution of dictionary values among the sublists of a list whenever a dictionary file is loaded into memory


Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/7e0584e7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/7e0584e7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/7e0584e7

Branch: refs/heads/master
Commit: 7e0584e7a1d90724e88fffd6fcea15e5ba640da8
Parents: 62c0b05
Author: manishgupt88 <to...@gmail.com>
Authored: Tue Jul 19 14:55:52 2016 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Tue Aug 23 18:08:16 2016 +0530

----------------------------------------------------------------------
 .../AbstractColumnDictionaryInfo.java           |  43 +++++---
 .../ColumnDictionaryChunkIterator.java          | 101 +++++++++++++++++++
 .../cache/dictionary/ColumnDictionaryInfo.java  |  47 ++++++++-
 .../dictionary/DictionaryCacheLoaderImpl.java   |  49 +++++++--
 .../core/cache/dictionary/DictionaryInfo.java   |   8 ++
 .../core/reader/CarbonDictionaryReader.java     |   5 +-
 .../core/reader/CarbonDictionaryReaderImpl.java |  43 ++++++--
 .../apache/carbondata/core/util/CarbonUtil.java |  19 ++++
 .../core/writer/CarbonDictionaryWriterImpl.java |  12 +--
 .../writer/CarbonDictionaryWriterImplTest.java  |   6 +-
 10 files changed, 283 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7e0584e7/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java
index a62695c..ad766d7 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java
@@ -25,6 +25,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
  * class that implements cacheable interface and methods specific to column dictionary
@@ -62,6 +63,11 @@ public abstract class AbstractColumnDictionaryInfo implements DictionaryInfo {
   private long dictionaryMetaFileLength;
 
   /**
+   * size of one dictionary bucket
+   */
+  private final int dictionaryOneChunkSize = CarbonUtil.getDictionaryChunkSize();
+
+  /**
    * This method will return the timestamp of file based on which decision
    * the decision will be taken whether to read that file or not
    *
@@ -99,6 +105,16 @@ public abstract class AbstractColumnDictionaryInfo implements DictionaryInfo {
   }
 
   /**
+   * This method will return the size of of last dictionary chunk so that only that many
+   * values are read from the dictionary reader
+   *
+   * @return size of last dictionary chunk
+   */
+  @Override public int getSizeOfLastDictionaryChunk() {
+    return 0;
+  }
+
+  /**
    * This method will decrement the access count for a column by 1
    * whenever a column usage is complete
    */
@@ -241,22 +257,19 @@ public abstract class AbstractColumnDictionaryInfo implements DictionaryInfo {
    */
   protected byte[] getDictionaryBytesFromSurrogate(int surrogateKey) {
     byte[] dictionaryValueInBytes = null;
-    int totalSizeOfDictionaryChunksTraversed = 0;
-    for (List<byte[]> oneDictionaryChunk : dictionaryChunks) {
-      totalSizeOfDictionaryChunksTraversed =
-          totalSizeOfDictionaryChunksTraversed + oneDictionaryChunk.size();
-      // skip the dictionary chunk till surrogate key is lesser than size of
-      // dictionary chunks traversed
-      if (totalSizeOfDictionaryChunksTraversed < surrogateKey) {
-        continue;
+    // surrogate key starts from 1 and list index will start from 0, so lets say if surrogate
+    // key is 10 then value will present at index 9 of the dictionary chunk list
+    int actualSurrogateIndex = surrogateKey - 1;
+    // lets say dictionaryOneChunkSize = 10, surrogateKey = 10, so bucket index will
+    // be 0 and dictionary chunk index will be 9 to get the value
+    int dictionaryBucketIndex = actualSurrogateIndex / dictionaryOneChunkSize;
+    if (dictionaryChunks.size() > dictionaryBucketIndex) {
+      int indexInsideBucket = actualSurrogateIndex % dictionaryOneChunkSize;
+      List<byte[]> dictionaryBucketContainingSurrogateValue =
+          dictionaryChunks.get(dictionaryBucketIndex);
+      if (dictionaryBucketContainingSurrogateValue.size() > indexInsideBucket) {
+        dictionaryValueInBytes = dictionaryBucketContainingSurrogateValue.get(indexInsideBucket);
       }
-      // lets say surrogateKey = 26, total size traversed is 28, dictionary chunk size = 12
-      // then surrogate position in dictionary chunk list is = 26 - (28-12) - 1 = 9
-      // -1 because list index starts from 0
-      int surrogatePositionInDictionaryChunk =
-          surrogateKey - (totalSizeOfDictionaryChunksTraversed - oneDictionaryChunk.size()) - 1;
-      dictionaryValueInBytes = oneDictionaryChunk.get(surrogatePositionInDictionaryChunk);
-      break;
     }
     return dictionaryValueInBytes;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7e0584e7/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIterator.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIterator.java
new file mode 100644
index 0000000..e87c146
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIterator.java
@@ -0,0 +1,101 @@
+/*
+ * 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.carbondata.core.cache.dictionary;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.format.ColumnDictionaryChunk;
+
+/**
+ * This class is a wrapper over column dictionary chunk thrift object.
+ * The wrapper class wraps the list<ColumnDictionaryChunk> and provides an API
+ * to fill the byte array into list
+ */
+public class ColumnDictionaryChunkIterator extends CarbonIterator {
+
+  /**
+   * list of dictionaryChunks
+   */
+  private List<ColumnDictionaryChunk> columnDictionaryChunks;
+
+  /**
+   * size of the list
+   */
+  private int size;
+
+  /**
+   * Current index of the list
+   */
+  private int currentSize;
+
+  /**
+   * variable holds the count of elements already iterated
+   */
+  private int iteratorIndex;
+
+  /**
+   * variable holds the current index of List<List<byte[]>> being traversed
+   */
+  private int outerIndex;
+
+  /**
+   * Constructor of ColumnDictionaryChunkIterator
+   *
+   * @param columnDictionaryChunks
+   */
+  public ColumnDictionaryChunkIterator(List<ColumnDictionaryChunk> columnDictionaryChunks) {
+    this.columnDictionaryChunks = columnDictionaryChunks;
+    for (ColumnDictionaryChunk dictionaryChunk : columnDictionaryChunks) {
+      this.size += dictionaryChunk.getValues().size();
+    }
+  }
+
+  /**
+   * Returns {@code true} if the iteration has more elements.
+   * (In other words, returns {@code true} if {@link #next} would
+   * return an element rather than throwing an exception.)
+   *
+   * @return {@code true} if the iteration has more elements
+   */
+  @Override public boolean hasNext() {
+    return (currentSize < size);
+  }
+
+  /**
+   * Returns the next element in the iteration.
+   * The method pics the next elements from the first inner list till first is not finished, pics
+   * the second inner list ...
+   *
+   * @return the next element in the iteration
+   */
+  @Override public byte[] next() {
+    if (iteratorIndex >= columnDictionaryChunks.get(outerIndex).getValues().size()) {
+      iteratorIndex = 0;
+      outerIndex++;
+    }
+    ByteBuffer buffer = columnDictionaryChunks.get(outerIndex).getValues().get(iteratorIndex);
+    byte[] value = buffer.array();
+    currentSize++;
+    iteratorIndex++;
+    return value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7e0584e7/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
index 08d9bef..b2d81da 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
@@ -30,6 +30,7 @@ import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
  * class that implements methods specific for dictionary data look up
@@ -112,10 +113,50 @@ public class ColumnDictionaryInfo extends AbstractColumnDictionaryInfo {
   /**
    * This method will add a new dictionary chunk to existing list of dictionary chunks
    *
-   * @param dictionaryChunk
+   * @param newDictionaryChunk
    */
-  @Override public void addDictionaryChunk(List<byte[]> dictionaryChunk) {
-    dictionaryChunks.add(dictionaryChunk);
+  @Override public void addDictionaryChunk(List<byte[]> newDictionaryChunk) {
+    if (dictionaryChunks.size() > 0) {
+      // Ensure that each time a new dictionary chunk is getting added to the
+      // dictionary chunks list, equal distribution of dictionary values should
+      // be there in the sublists of dictionary chunk list
+      List<byte[]> lastDictionaryChunk = dictionaryChunks.get(dictionaryChunks.size() - 1);
+      int dictionaryOneChunkSize = CarbonUtil.getDictionaryChunkSize();
+      int differenceInLastDictionaryAndOneChunkSize =
+          dictionaryOneChunkSize - lastDictionaryChunk.size();
+      if (differenceInLastDictionaryAndOneChunkSize > 0) {
+        // if difference is greater than new dictionary size then copy a part of list
+        // else copy the complete new dictionary chunk list in the last dictionary chunk list
+        if (differenceInLastDictionaryAndOneChunkSize >= newDictionaryChunk.size()) {
+          lastDictionaryChunk.addAll(newDictionaryChunk);
+        } else {
+          List<byte[]> subListOfNewDictionaryChunk =
+              newDictionaryChunk.subList(0, differenceInLastDictionaryAndOneChunkSize);
+          lastDictionaryChunk.addAll(subListOfNewDictionaryChunk);
+          List<byte[]> remainingNewDictionaryChunk = newDictionaryChunk
+              .subList(differenceInLastDictionaryAndOneChunkSize, newDictionaryChunk.size());
+          dictionaryChunks.add(remainingNewDictionaryChunk);
+        }
+      } else {
+        dictionaryChunks.add(newDictionaryChunk);
+      }
+    } else {
+      dictionaryChunks.add(newDictionaryChunk);
+    }
+  }
+
+  /**
+   * This method will return the size of of last dictionary chunk so that only that many
+   * values are read from the dictionary reader
+   *
+   * @return size of last dictionary chunk
+   */
+  @Override public int getSizeOfLastDictionaryChunk() {
+    int lastDictionaryChunkSize = 0;
+    if (dictionaryChunks.size() > 0) {
+      lastDictionaryChunkSize = dictionaryChunks.get(dictionaryChunks.size() - 1).size();
+    }
+    return lastDictionaryChunkSize;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7e0584e7/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
index 6e603f9..e798c01 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
@@ -20,14 +20,18 @@
 package org.apache.carbondata.core.cache.dictionary;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.carbondata.common.factory.CarbonCommonFactory;
 import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
 import org.apache.carbondata.core.carbon.ColumnIdentifier;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.reader.CarbonDictionaryReader;
 import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
 import org.apache.carbondata.core.service.DictionaryService;
+import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
  * This class is responsible for loading the dictionary data for given columns
@@ -71,12 +75,44 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
   @Override public void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier,
       long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
       throws IOException {
-    List<byte[]> dictionaryChunk =
+    Iterator<byte[]> columnDictionaryChunkWrapper =
         load(columnIdentifier, dictionaryChunkStartOffset, dictionaryChunkEndOffset);
     if (loadSortIndex) {
       readSortIndexFile(dictionaryInfo, columnIdentifier);
     }
-    dictionaryInfo.addDictionaryChunk(dictionaryChunk);
+    fillDictionaryValuesAndAddToDictionaryChunks(dictionaryInfo, columnDictionaryChunkWrapper);
+  }
+
+  /**
+   * This method will fill the dictionary values according to dictionary bucket size and
+   * add to the dictionary chunk list
+   *
+   * @param dictionaryInfo
+   * @param columnDictionaryChunkWrapper
+   */
+  private void fillDictionaryValuesAndAddToDictionaryChunks(DictionaryInfo dictionaryInfo,
+      Iterator<byte[]> columnDictionaryChunkWrapper) {
+    int dictionaryChunkSize = CarbonUtil.getDictionaryChunkSize();
+    int sizeOfLastDictionaryChunk = dictionaryInfo.getSizeOfLastDictionaryChunk();
+    int sizeOfOneDictionaryChunk = dictionaryChunkSize - sizeOfLastDictionaryChunk;
+    if (sizeOfOneDictionaryChunk == 0) {
+      sizeOfOneDictionaryChunk = dictionaryChunkSize;
+    }
+    List<List<byte[]>> dictionaryChunks =
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    List<byte[]> oneDictionaryChunk = new ArrayList<>(sizeOfOneDictionaryChunk);
+    dictionaryChunks.add(oneDictionaryChunk);
+    while (columnDictionaryChunkWrapper.hasNext()) {
+      oneDictionaryChunk.add(columnDictionaryChunkWrapper.next());
+      if (oneDictionaryChunk.size() >= sizeOfOneDictionaryChunk) {
+        sizeOfOneDictionaryChunk = dictionaryChunkSize;
+        oneDictionaryChunk = new ArrayList<>(sizeOfOneDictionaryChunk);
+        dictionaryChunks.add(oneDictionaryChunk);
+      }
+    }
+    for (List<byte[]> dictionaryChunk : dictionaryChunks) {
+      dictionaryInfo.addDictionaryChunk(dictionaryChunk);
+    }
   }
 
   /**
@@ -85,19 +121,18 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    * @param columnIdentifier column unique identifier
    * @param startOffset      start offset of dictionary file
    * @param endOffset        end offset of dictionary file
-   * @return list of dictionary value
+   * @return iterator over dictionary values
    * @throws IOException
    */
-  private List<byte[]> load(ColumnIdentifier columnIdentifier, long startOffset, long endOffset)
+  private Iterator<byte[]> load(ColumnIdentifier columnIdentifier, long startOffset, long endOffset)
       throws IOException {
     CarbonDictionaryReader dictionaryReader = getDictionaryReader(columnIdentifier);
-    List<byte[]> dictionaryValue = null;
     try {
-      dictionaryValue = dictionaryReader.read(startOffset, endOffset);
+      Iterator<byte[]> columnDictionaryChunkWrapper = dictionaryReader.read(startOffset, endOffset);
+      return columnDictionaryChunkWrapper;
     } finally {
       dictionaryReader.close();
     }
-    return dictionaryValue;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7e0584e7/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java
index e34860a..a191545 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java
@@ -58,6 +58,14 @@ public interface DictionaryInfo extends Cacheable, Dictionary {
   void addDictionaryChunk(List<byte[]> dictionaryChunk);
 
   /**
+   * This method will return the size of of last dictionary chunk so that only that many
+   * values are read from the dictionary reader
+   *
+   * @return size of last dictionary chunk
+   */
+  int getSizeOfLastDictionaryChunk();
+
+  /**
    * This method will set the sort order index of a dictionary column.
    * Sort order index if the index of dictionary values after they are sorted.
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7e0584e7/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReader.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReader.java
index dded6c2..a2289bd 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReader.java
@@ -21,6 +21,7 @@ package org.apache.carbondata.core.reader;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Iterator;
 import java.util.List;
 
 /**
@@ -63,8 +64,8 @@ public interface CarbonDictionaryReader extends Closeable {
    *
    * @param startOffset start offset of dictionary file
    * @param endOffset   end offset of dictionary file
-   * @return list of byte array. Each byte array is unique dictionary value
+   * @return iterator over byte array. Each byte array is unique dictionary value
    * @throws IOException if an I/O error occurs
    */
-  List<byte[]> read(long startOffset, long endOffset) throws IOException;
+  Iterator<byte[]> read(long startOffset, long endOffset) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7e0584e7/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
index a843701..d3f9f31 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
@@ -22,13 +22,14 @@ package org.apache.carbondata.core.reader;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.carbondata.common.factory.CarbonCommonFactory;
+import org.apache.carbondata.core.cache.dictionary.ColumnDictionaryChunkIterator;
 import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
 import org.apache.carbondata.core.carbon.ColumnIdentifier;
 import org.apache.carbondata.core.carbon.path.CarbonTablePath;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.service.PathService;
 import org.apache.carbondata.format.ColumnDictionaryChunk;
 
@@ -115,7 +116,9 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
         carbonDictionaryColumnMetaChunks.get(carbonDictionaryColumnMetaChunks.size() - 1);
     // end offset till where the dictionary file has to be read
     long endOffset = carbonDictionaryColumnMetaChunk.getEnd_offset();
-    return read(carbonDictionaryColumnMetaChunks, startOffset, endOffset);
+    List<ColumnDictionaryChunk> columnDictionaryChunks =
+        read(carbonDictionaryColumnMetaChunks, startOffset, endOffset);
+    return getDictionaryList(columnDictionaryChunks);
   }
 
   /**
@@ -126,13 +129,17 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
    *
    * @param startOffset start offset of dictionary file
    * @param endOffset   end offset of dictionary file
-   * @return list of byte array. Each byte array is unique dictionary value
+   * @return iterator over byte array. Each byte array is unique dictionary value
    * @throws IOException if an I/O error occurs
    */
-  @Override public List<byte[]> read(long startOffset, long endOffset) throws IOException {
+  @Override public Iterator<byte[]> read(long startOffset, long endOffset) throws IOException {
     List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks =
         readDictionaryMetadataFile();
-    return read(carbonDictionaryColumnMetaChunks, startOffset, endOffset);
+    List<ColumnDictionaryChunk> columnDictionaryChunks =
+        read(carbonDictionaryColumnMetaChunks, startOffset, endOffset);
+    Iterator<byte[]> columnDictionaryChunkWrapper =
+        new ColumnDictionaryChunkIterator(columnDictionaryChunks);
+    return columnDictionaryChunkWrapper;
   }
 
   /**
@@ -154,11 +161,12 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
    * @param startOffset                      start offset for dictionary data file
    * @param endOffset                        end offset till where data has
    *                                         to be read from dictionary data file
-   * @return list of byte array dictionary values
+   * @return list of byte column dictionary values
    * @throws IOException readDictionary file method throws IO exception
    */
-  private List<byte[]> read(List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks,
-      long startOffset, long endOffset) throws IOException {
+  private List<ColumnDictionaryChunk> read(
+      List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks, long startOffset,
+      long endOffset) throws IOException {
     // calculate the number of chunks to be read from dictionary file from start offset
     int dictionaryChunkCountsToBeRead =
         calculateTotalDictionaryChunkCountsToBeRead(carbonDictionaryColumnMetaChunks, startOffset,
@@ -168,9 +176,22 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
     // read the required number of chunks from dictionary file
     List<ColumnDictionaryChunk> columnDictionaryChunks =
         readDictionaryFile(startOffset, dictionaryChunkCountsToBeRead);
-    // convert byte buffer list to byte array list of dictionary vlaues
-    List<byte[]> dictionaryValues =
-        new ArrayList<byte[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    return columnDictionaryChunks;
+  }
+
+  /**
+   * This method will put all the dictionary chunks into one list and return that list
+   *
+   * @param columnDictionaryChunks
+   * @return
+   */
+  private List<byte[]> getDictionaryList(List<ColumnDictionaryChunk> columnDictionaryChunks) {
+    int dictionaryListSize = 0;
+    for (ColumnDictionaryChunk dictionaryChunk : columnDictionaryChunks) {
+      dictionaryListSize = dictionaryListSize + dictionaryChunk.getValues().size();
+    }
+    // convert byte buffer list to byte array list of dictionary values
+    List<byte[]> dictionaryValues = new ArrayList<byte[]>(dictionaryListSize);
     for (ColumnDictionaryChunk dictionaryChunk : columnDictionaryChunks) {
       convertAndFillByteBufferListToByteArrayList(dictionaryValues, dictionaryChunk.getValues());
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7e0584e7/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 7acabf2..169eaf8 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -1424,5 +1424,24 @@ public final class CarbonUtil {
     }
   }
 
+  /**
+   * initialize the value of dictionary chunk that can be kept in memory at a time
+   *
+   * @return
+   */
+  public static int getDictionaryChunkSize() {
+    int dictionaryOneChunkSize = 0;
+    try {
+      dictionaryOneChunkSize = Integer.parseInt(CarbonProperties.getInstance()
+          .getProperty(CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE,
+              CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE_DEFAULT));
+    } catch (NumberFormatException e) {
+      dictionaryOneChunkSize =
+          Integer.parseInt(CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE_DEFAULT);
+      LOGGER.error("Dictionary chunk size not configured properly. Taking default size "
+          + dictionaryOneChunkSize);
+    }
+    return dictionaryOneChunkSize;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7e0584e7/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
index 2e08610..316832d 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
@@ -38,7 +38,6 @@ import org.apache.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReader;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
 import org.apache.carbondata.core.service.PathService;
-import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.format.ColumnDictionaryChunk;
 import org.apache.carbondata.format.ColumnDictionaryChunkMeta;
@@ -263,16 +262,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
    * initialize the value of dictionary chunk that can be kept in memory at a time
    */
   private void initDictionaryChunkSize() {
-    try {
-      dictionary_one_chunk_size = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE,
-              CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE_DEFAULT));
-    } catch (NumberFormatException e) {
-      dictionary_one_chunk_size =
-          Integer.parseInt(CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE_DEFAULT);
-      LOGGER.error("Dictionary chunk size not configured properly. Taking default size "
-              + dictionary_one_chunk_size);
-    }
+    dictionary_one_chunk_size = CarbonUtil.getDictionaryChunkSize();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7e0584e7/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java b/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
index 8d7653a..5c80bef 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
@@ -28,6 +28,7 @@ import java.net.URISyntaxException;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 import java.util.UUID;
@@ -457,7 +458,10 @@ public class CarbonDictionaryWriterImplTest {
       if (0 == dictionaryEndOffset) {
         dictionaryValues = dictionaryReader.read(dictionaryStartOffset);
       } else {
-        dictionaryValues = dictionaryReader.read(dictionaryStartOffset, dictionaryEndOffset);
+        Iterator<byte[]> itr = dictionaryReader.read(dictionaryStartOffset, dictionaryEndOffset);
+        while (itr.hasNext()) {
+          dictionaryValues.add(itr.next());
+        }
       }
     } finally {
       dictionaryReader.close();


[2/2] incubator-carbondata git commit: [CARBONDATA-80] This closes #44

Posted by gv...@apache.org.
[CARBONDATA-80] This closes #44


Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/2d4609cd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/2d4609cd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/2d4609cd

Branch: refs/heads/master
Commit: 2d4609cdface93ea3f3a7a92e088e5b98f24f7e2
Parents: 62c0b05 7e0584e
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Tue Aug 23 19:32:03 2016 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Tue Aug 23 19:32:03 2016 +0530

----------------------------------------------------------------------
 .../AbstractColumnDictionaryInfo.java           |  43 +++++---
 .../ColumnDictionaryChunkIterator.java          | 101 +++++++++++++++++++
 .../cache/dictionary/ColumnDictionaryInfo.java  |  47 ++++++++-
 .../dictionary/DictionaryCacheLoaderImpl.java   |  49 +++++++--
 .../core/cache/dictionary/DictionaryInfo.java   |   8 ++
 .../core/reader/CarbonDictionaryReader.java     |   5 +-
 .../core/reader/CarbonDictionaryReaderImpl.java |  43 ++++++--
 .../apache/carbondata/core/util/CarbonUtil.java |  19 ++++
 .../core/writer/CarbonDictionaryWriterImpl.java |  12 +--
 .../writer/CarbonDictionaryWriterImplTest.java  |   6 +-
 10 files changed, 283 insertions(+), 50 deletions(-)
----------------------------------------------------------------------