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/f82ad267 Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/f82ad267 Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/f82ad267 Branch: refs/heads/branch-0.1 Commit: f82ad267ea57321a5bb6cff983bd507270f2bd38 Parents: 2b14960 Author: manishgupt88 <tomanishgupt...@gmail.com> Authored: Tue Jul 19 14:55:52 2016 +0530 Committer: ravipesala <ravi.pes...@gmail.com> Committed: Thu Sep 22 09:27:51 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/f82ad267/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/f82ad267/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/f82ad267/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/f82ad267/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/f82ad267/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/f82ad267/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/f82ad267/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/f82ad267/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 c028efd..2460f6e 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 @@ -1366,5 +1366,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/f82ad267/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/f82ad267/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();