[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-27 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/carbondata/pull/2402


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-25 Thread ravipesala
Github user ravipesala commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197821950
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
 ---
@@ -110,44 +135,51 @@ public 
CarbonFactDataWriterImplV3(CarbonFactDataHandlerModel model) {
*/
   @Override public void writeTablePage(TablePage tablePage)
   throws CarbonDataWriterException,IOException {
-// condition for writting all the pages
-if (!tablePage.isLastPage()) {
-  boolean isAdded = false;
-  // check if size more than blocklet size then write the page to file
-  if (blockletDataHolder.getSize() + 
tablePage.getEncodedTablePage().getEncodedSize() >=
-  blockletSizeThreshold) {
-// if blocklet size exceeds threshold, write blocklet data
-if (blockletDataHolder.getEncodedTablePages().size() == 0) {
-  isAdded = true;
-  addPageData(tablePage);
-}
+try {
--- End diff --

dnt format the code if code is not changed


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-25 Thread ravipesala
Github user ravipesala commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197821658
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
 ---
@@ -76,7 +79,29 @@ public 
CarbonFactDataWriterImplV3(CarbonFactDataHandlerModel model) {
   blockletSizeThreshold = fileSizeInBytes;
   LOGGER.info("Blocklet size configure for table is: " + 
blockletSizeThreshold);
 }
-blockletDataHolder = new BlockletDataHolder();
+int numberOfCores = CarbonProperties.getInstance().getNumberOfCores();
--- End diff --

please remove unused code


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-25 Thread ravipesala
Github user ravipesala commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197820697
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/localdictionary/PageLevelDictionary.java
 ---
@@ -0,0 +1,118 @@
+/*
+ * 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.localdictionary;
+
+import java.io.IOException;
+import java.util.BitSet;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.ColumnType;
+import org.apache.carbondata.core.datastore.TableSpec;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import 
org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoder;
+import 
org.apache.carbondata.core.datastore.page.encoding.compress.DirectCompressCodec;
+import 
org.apache.carbondata.core.datastore.page.statistics.DummyStatsCollector;
+import 
org.apache.carbondata.core.localdictionary.exception.DictionaryThresholdReachedException;
+import 
org.apache.carbondata.core.localdictionary.generator.LocalDictionaryGenerator;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.format.LocalDictionaryChunk;
+
+/**
+ * Class to maintain page level dictionary. It will store all unique 
dictionary values
+ * used in a page. This is required while writing blocklet level 
dictionary in carbondata
+ * file
+ */
+public class PageLevelDictionary {
+
+  /**
+   * dictionary generator to generate dictionary values for page data
+   */
+  private LocalDictionaryGenerator localDictionaryGenerator;
+
+  /**
+   * set of dictionary surrogate key in this page
+   */
+  private BitSet usedDictionaryValues;
+
+  private int maxDictValue;
+
+  private String columnName;
+
+  public PageLevelDictionary(LocalDictionaryGenerator 
localDictionaryGenerator,String columnName) {
+this.localDictionaryGenerator = localDictionaryGenerator;
+this.usedDictionaryValues = new BitSet();
+this.columnName = columnName;
+  }
+
+  /**
+   * Below method will be used to get the dictionary value
+   *
+   * @param data column data
+   * @return dictionary value
+   * @throws DictionaryThresholdReachedException when threshold crossed 
for column
+   */
+  public int getDictionaryValue(byte[] data) throws 
DictionaryThresholdReachedException {
+int dictionaryValue = 
localDictionaryGenerator.generateDictionary(data);
+this.usedDictionaryValues.set(dictionaryValue);
+if (maxDictValue < dictionaryValue) {
+  maxDictValue = dictionaryValue;
+}
+return dictionaryValue;
+  }
+
+  /**
+   * Method to merge the dictionary value across pages
+   *
+   * @param pageLevelDictionary other page level dictionary
+   */
+  public void mergerDictionaryValues(PageLevelDictionary 
pageLevelDictionary) {
+usedDictionaryValues.and(pageLevelDictionary.usedDictionaryValues);
+  }
+
+  /**
+   * Below method will be used to get the local dictionary chunk for 
writing
+   * @TODO Support for numeric data type dictionary exclude columns
+   * @return encoded local dictionary chunk
+   * @throws MemoryException
+   * in case of problem in encoding
+   * @throws IOException
+   * in case of problem in encoding
+   */
+  public LocalDictionaryChunk getLocalDictionaryChunkForBlocklet()
+  throws MemoryException, IOException {
+// TODO support for actual data type dictionary ColumnSPEC
+TableSpec.ColumnSpec spec = TableSpec.ColumnSpec
+.newInstance(columnName, DataTypes.BYTE_ARRAY, 
ColumnType.PLAIN_VALUE);
+ColumnPage dictionaryColumnPage = ColumnPage.newPage(spec, 
DataTypes.BYTE_ARRAY, maxDictValue);
+// TODO support data type specific stats collector for numeric data 
types
+

[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-25 Thread ravipesala
Github user ravipesala commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197817666
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/localdictionary/dictionaryholder/MapBasedDictionaryStore.java
 ---
@@ -0,0 +1,135 @@
+/*
+ * 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.localdictionary.dictionaryholder;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import 
org.apache.carbondata.core.cache.dictionary.DictionaryByteArrayWrapper;
+import 
org.apache.carbondata.core.localdictionary.exception.DictionaryThresholdReachedException;
+
+/**
+ * Map based dictionary holder class, it will use map to hold
+ * the dictionary key and its value
+ */
+public class MapBasedDictionaryStore implements DictionaryStore {
+
+  /**
+   * use to assign dictionary value to new key
+   */
+  private int lastAssignValue;
+
+  /**
+   * to maintain dictionary key value
+   */
+  private final Map dictionary;
+
+  /**
+   * maintaining array for reverse lookup
+   * otherwise iterating everytime in map for reverse lookup will be 
slowdown the performance
+   * It will only maintain the reference
+   */
+  private byte[][] referenceDictionaryArray;
--- End diff --

Better directly use `DictionaryByteArrayWrapper` array here


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-25 Thread ravipesala
Github user ravipesala commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197813935
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
 ---
@@ -623,5 +640,83 @@ public DataMapWriterListener 
getDataMapWriterlistener() {
 return dataMapWriterlistener;
   }
 
+  public Map getColumnLocalDictGenMap() {
+return columnLocalDictGenMap;
+  }
+
+  /**
+   * This method prepares a map which will have column and local 
dictionary generator mapping for
+   * all the local dictionary columns.
+   * @param carbonTable
+   * @param wrapperColumnSchema
+   * @param carbonFactDataHandlerModel
+   */
+  public static void setLocalDictToModel(CarbonTable carbonTable,
--- End diff --

Keep as `private`


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-25 Thread ravipesala
Github user ravipesala commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197813606
  
--- Diff: 
core/src/test/java/org/apache/carbondata/core/util/CarbonMetadataUtilTest.java 
---
@@ -172,71 +172,71 @@
 IndexHeader indexheaderResult = getIndexHeader(columnCardinality, 
columnSchemaList, 0, 0L);
 assertEquals(indexHeader, indexheaderResult);
   }
-
-  @Test public void testConvertFileFooter() throws Exception {
-int[] cardinality = { 1, 2, 3, 4, 5 };
-
-org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema 
colSchema =
-new 
org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema();
-org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema 
colSchema1 =
-new 
org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema();
-
List
-columnSchemaList = new ArrayList<>();
-columnSchemaList.add(colSchema);
-columnSchemaList.add(colSchema1);
-
-SegmentProperties segmentProperties = new 
SegmentProperties(columnSchemaList, cardinality);
-
-final EncodedColumnPage measure = new EncodedColumnPage(new 
DataChunk2(), new byte[]{0,1},
-PrimitivePageStatsCollector.newInstance(
-org.apache.carbondata.core.metadata.datatype.DataTypes.BYTE));
-new MockUp() {
-  @SuppressWarnings("unused") @Mock
-  public EncodedColumnPage getMeasure(int measureIndex) {
-return measure;
-  }
-};
-
-new MockUp() {
-  @SuppressWarnings("unused") @Mock
-  public byte[] serializeStartKey() {
-return new byte[]{1, 2};
-  }
-
-  @SuppressWarnings("unused") @Mock
-  public byte[] serializeEndKey() {
-return new byte[]{1, 2};
-  }
-};
-
-TablePageKey key = new TablePageKey(3, segmentProperties, false);
-EncodedTablePage encodedTablePage = EncodedTablePage.newInstance(3, 
new EncodedColumnPage[0], new EncodedColumnPage[0],
-key);
-
-List encodedTablePageList = new ArrayList<>();
-encodedTablePageList.add(encodedTablePage);
-
-BlockletInfo3 blockletInfoColumnar1 = new BlockletInfo3();
-
-List blockletInfoColumnarList = new ArrayList<>();
-blockletInfoColumnarList.add(blockletInfoColumnar1);
-
-byte[] byteMaxArr = "1".getBytes();
-byte[] byteMinArr = "2".getBytes();
-
-BlockletIndex index = getBlockletIndex(encodedTablePageList, 
segmentProperties.getMeasures());
-List indexList = new ArrayList<>();
-indexList.add(index);
-
-BlockletMinMaxIndex blockletMinMaxIndex = new BlockletMinMaxIndex();
-blockletMinMaxIndex.addToMax_values(ByteBuffer.wrap(byteMaxArr));
-blockletMinMaxIndex.addToMin_values(ByteBuffer.wrap(byteMinArr));
-FileFooter3 footer = 
convertFileFooterVersion3(blockletInfoColumnarList,
-indexList,
-cardinality, 2);
-assertEquals(footer.getBlocklet_index_list(), indexList);
-
-  }
+//
--- End diff --

remove if not required


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-25 Thread ravipesala
Github user ravipesala commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197807784
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/blocklet/BlockletEncodedColumnPage.java
 ---
@@ -0,0 +1,187 @@
+/*
+ * 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.datastore.blocklet;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.page.FallbackColumnPageEncoder;
+import org.apache.carbondata.core.datastore.page.FallbackEncodedColumnPage;
+import 
org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
+import org.apache.carbondata.core.localdictionary.PageLevelDictionary;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.format.LocalDictionaryChunk;
+
+/**
+ * Maintains the list of encoded page of a column in a blocklet
+ * and encoded dictionary values only if column is encoded using local
+ * dictionary
+ * Handle the fallback if all the pages in blocklet are not
+ * encoded with local dictionary
+ */
+public class BlockletEncodedColumnPage {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+  
LogServiceFactory.getLogService(BlockletEncodedColumnPage.class.getName());
+
+  /**
+   * list of encoded page of a column in a blocklet
+   */
+  private List encodedColumnPageList;
+
+  /**
+   * fallback executor service
+   */
+  private ExecutorService fallbackExecutorService;
+
+  /**
+   * to check whether pages are local dictionary encoded or not
+   */
+  private boolean isLocalDictEncoded;
+
+  /**
+   * page level dictionary only when column is encoded with local 
dictionary
+   */
+  private PageLevelDictionary pageLevelDictionary;
+
+  /**
+   * fallback future task queue;
+   */
+  private ArrayDeque> 
fallbackFutureQueue;
+
+  BlockletEncodedColumnPage(ExecutorService fallbackExecutorService,
+  EncodedColumnPage encodedColumnPage) {
+this.encodedColumnPageList = new ArrayList<>();
+this.fallbackExecutorService = fallbackExecutorService;
+this.encodedColumnPageList.add(encodedColumnPage);
+// if dimension page is local dictionary enabled and encoded with 
local dictionary
+if (encodedColumnPage.isLocalDictionaryEnabled() && encodedColumnPage
+.isLocalDictGeneratedPage()) {
+  this.isLocalDictEncoded = true;
+  // get first page dictionary
+  this.pageLevelDictionary = encodedColumnPage.getPageDictionary();
+}
+  }
+
+  /**
+   * Below method will be used to add column page of a column
+   *
+   * @param encodedColumnPage
+   * encoded column page
+   * @throws ExecutionException
+   * failure in fallback
+   * @throws InterruptedException
+   * failure during fallback
+   */
+  void addEncodedColumnColumnPage(EncodedColumnPage encodedColumnPage)
+  throws ExecutionException, InterruptedException {
+// if local dictionary is false or column is encoded with local 
dictionary then
+// add a page
+if (!isLocalDictEncoded || 
encodedColumnPage.isLocalDictGeneratedPage()) {
+  this.encodedColumnPageList.add(encodedColumnPage);
+  // merge page level dictionary values
+  if (null != this.pageLevelDictionary) {
+
pageLevelDictionary.mergerDictionaryValues(encodedColumnPage.getPageDictionary());
+  }
+} else {
+  // if older pages were encoded with dictionary and new pages are 
without 

[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-25 Thread ravipesala
Github user ravipesala commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197804388
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/blocklet/BlockletEncodedColumnPage.java
 ---
@@ -0,0 +1,187 @@
+/*
+ * 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.datastore.blocklet;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.page.FallbackColumnPageEncoder;
+import org.apache.carbondata.core.datastore.page.FallbackEncodedColumnPage;
+import 
org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
+import org.apache.carbondata.core.localdictionary.PageLevelDictionary;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.format.LocalDictionaryChunk;
+
+/**
+ * Maintains the list of encoded page of a column in a blocklet
+ * and encoded dictionary values only if column is encoded using local
+ * dictionary
+ * Handle the fallback if all the pages in blocklet are not
+ * encoded with local dictionary
+ */
+public class BlockletEncodedColumnPage {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+  
LogServiceFactory.getLogService(BlockletEncodedColumnPage.class.getName());
+
+  /**
+   * list of encoded page of a column in a blocklet
+   */
+  private List encodedColumnPageList;
+
+  /**
+   * fallback executor service
+   */
+  private ExecutorService fallbackExecutorService;
+
+  /**
+   * to check whether pages are local dictionary encoded or not
+   */
+  private boolean isLocalDictEncoded;
+
+  /**
+   * page level dictionary only when column is encoded with local 
dictionary
+   */
+  private PageLevelDictionary pageLevelDictionary;
+
+  /**
+   * fallback future task queue;
+   */
+  private ArrayDeque> 
fallbackFutureQueue;
+
+  BlockletEncodedColumnPage(ExecutorService fallbackExecutorService,
+  EncodedColumnPage encodedColumnPage) {
--- End diff --

Don't add `encodedColumnPage` from constructor, use 
`addEncodedColumnColumnPage`


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-25 Thread ravipesala
Github user ravipesala commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197803602
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/blocklet/BlockletEncodedColumnPage.java
 ---
@@ -0,0 +1,187 @@
+/*
+ * 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.datastore.blocklet;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.page.FallbackColumnPageEncoder;
+import org.apache.carbondata.core.datastore.page.FallbackEncodedColumnPage;
+import 
org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
+import org.apache.carbondata.core.localdictionary.PageLevelDictionary;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.format.LocalDictionaryChunk;
+
+/**
+ * Maintains the list of encoded page of a column in a blocklet
+ * and encoded dictionary values only if column is encoded using local
+ * dictionary
+ * Handle the fallback if all the pages in blocklet are not
+ * encoded with local dictionary
+ */
+public class BlockletEncodedColumnPage {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+  
LogServiceFactory.getLogService(BlockletEncodedColumnPage.class.getName());
+
+  /**
+   * list of encoded page of a column in a blocklet
+   */
+  private List encodedColumnPageList;
+
+  /**
+   * fallback executor service
+   */
+  private ExecutorService fallbackExecutorService;
+
+  /**
+   * to check whether pages are local dictionary encoded or not
+   */
+  private boolean isLocalDictEncoded;
+
+  /**
+   * page level dictionary only when column is encoded with local 
dictionary
+   */
+  private PageLevelDictionary pageLevelDictionary;
+
+  /**
+   * fallback future task queue;
+   */
+  private ArrayDeque> 
fallbackFutureQueue;
+
+  BlockletEncodedColumnPage(ExecutorService fallbackExecutorService,
+  EncodedColumnPage encodedColumnPage) {
+this.encodedColumnPageList = new ArrayList<>();
+this.fallbackExecutorService = fallbackExecutorService;
+this.encodedColumnPageList.add(encodedColumnPage);
+// if dimension page is local dictionary enabled and encoded with 
local dictionary
+if (encodedColumnPage.isLocalDictionaryEnabled() && encodedColumnPage
--- End diff --

Just keep `this.isLocalDictEncoded 
=encodedColumnPage.isLocalDictGeneratedPage()` should be ok


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-24 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197630595
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ---
@@ -482,7 +482,7 @@ public String getTableUniqueName() {
* @return
*/
   public boolean isLocalDictionaryEnabled() {
-return isLocalDictionaryEnabled;
--- End diff --

Currently Query part is not handled, so query will fail if local dictionary 
is enabled. I am working on the same


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-24 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197630599
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/page/LocalDictColumnPage.java
 ---
@@ -0,0 +1,319 @@
+/*
+ * 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.datastore.page;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.localdictionary.PageLevelDictionary;
+import 
org.apache.carbondata.core.localdictionary.exception.DictionaryThresholdReachedException;
+import 
org.apache.carbondata.core.localdictionary.generator.LocalDictionaryGenerator;
+import org.apache.carbondata.core.util.ByteUtil;
+
+/**
+ * Column page implementation for Local dictionary generated columns
+ * Its a decorator over two column page
+ * 1. Which will hold the actual data
+ * 2. Which will hold the dictionary encoded data
+ */
+public class LocalDictColumnPage extends ColumnPage {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+  LogServiceFactory.getLogService(LocalDictColumnPage.class.getName());
+
+  /**
+   * to maintain page level dictionary for column page
+   */
+  private PageLevelDictionary pageLevelDictionary;
+
+  /**
+   * to hold the actual data of the column
+   */
+  private ColumnPage actualDataColumnPage;
+
+  /**
+   * to hold the dictionary encoded column page
+   */
+  private ColumnPage encodedDataColumnPage;
+
+  /**
+   * to check if actual column page memory is already clear
+   */
+  private boolean isActualPageMemoryFreed;
+
+  /**
+   * Create a new column page with input data type and page size.
+   */
+  protected LocalDictColumnPage(ColumnPage actualDataColumnPage, 
ColumnPage encodedColumnpage,
+  LocalDictionaryGenerator localDictionaryGenerator) {
+super(actualDataColumnPage.getColumnSpec(), 
actualDataColumnPage.getDataType(),
+actualDataColumnPage.getPageSize());
+if (!localDictionaryGenerator.isThresholdReached()) {
+  pageLevelDictionary = new 
PageLevelDictionary(localDictionaryGenerator,
+  actualDataColumnPage.getColumnSpec().getFieldName());
+  this.encodedDataColumnPage = encodedColumnpage;
+}
+this.actualDataColumnPage = actualDataColumnPage;
+  }
+
+  @Override public byte[][] getByteArrayPage() {
+if (null != pageLevelDictionary) {
+  return encodedDataColumnPage.getByteArrayPage();
+} else {
+  return actualDataColumnPage.getByteArrayPage();
+}
+  }
+
+  /**
+   * Below method will be used to check whether page is local dictionary
+   * generated or not. This will be used for while enoding the the page
+   *
+   * @return
+   */
+  public boolean isLocalDictGeneratedPage() {
+return null != pageLevelDictionary;
+  }
+
+  /**
+   * Below method will be used to add column data to page
+   *
+   * @param rowId row number
+   * @param bytes actual data
+   */
+  @Override public void putBytes(int rowId, byte[] bytes) {
+if (null != pageLevelDictionary) {
+  try {
+actualDataColumnPage.putBytes(rowId, bytes);
+int dictionaryValue = 
pageLevelDictionary.getDictionaryValue(bytes);
+encodedDataColumnPage.putBytes(rowId, 
ByteUtil.toBytes(dictionaryValue));
+  } catch (DictionaryThresholdReachedException e) {
+LOGGER.error(e, "Local Dictionary threshold reached for the 
column: " + actualDataColumnPage
+.getColumnSpec().getFieldName());
+pageLevelDictionary = null;
+encodedDataColumnPage.freeMemory();
+encodedDataColumnPage = null;
+  }
+} 

[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-23 Thread akashrn5
Github user akashrn5 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197603924
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/page/LocalDictColumnPage.java
 ---
@@ -0,0 +1,319 @@
+/*
+ * 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.datastore.page;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.localdictionary.PageLevelDictionary;
+import 
org.apache.carbondata.core.localdictionary.exception.DictionaryThresholdReachedException;
+import 
org.apache.carbondata.core.localdictionary.generator.LocalDictionaryGenerator;
+import org.apache.carbondata.core.util.ByteUtil;
+
+/**
+ * Column page implementation for Local dictionary generated columns
+ * Its a decorator over two column page
+ * 1. Which will hold the actual data
+ * 2. Which will hold the dictionary encoded data
+ */
+public class LocalDictColumnPage extends ColumnPage {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+  LogServiceFactory.getLogService(LocalDictColumnPage.class.getName());
+
+  /**
+   * to maintain page level dictionary for column page
+   */
+  private PageLevelDictionary pageLevelDictionary;
+
+  /**
+   * to hold the actual data of the column
+   */
+  private ColumnPage actualDataColumnPage;
+
+  /**
+   * to hold the dictionary encoded column page
+   */
+  private ColumnPage encodedDataColumnPage;
+
+  /**
+   * to check if actual column page memory is already clear
+   */
+  private boolean isActualPageMemoryFreed;
+
+  /**
+   * Create a new column page with input data type and page size.
+   */
+  protected LocalDictColumnPage(ColumnPage actualDataColumnPage, 
ColumnPage encodedColumnpage,
+  LocalDictionaryGenerator localDictionaryGenerator) {
+super(actualDataColumnPage.getColumnSpec(), 
actualDataColumnPage.getDataType(),
+actualDataColumnPage.getPageSize());
+if (!localDictionaryGenerator.isThresholdReached()) {
+  pageLevelDictionary = new 
PageLevelDictionary(localDictionaryGenerator,
+  actualDataColumnPage.getColumnSpec().getFieldName());
+  this.encodedDataColumnPage = encodedColumnpage;
+}
+this.actualDataColumnPage = actualDataColumnPage;
+  }
+
+  @Override public byte[][] getByteArrayPage() {
+if (null != pageLevelDictionary) {
+  return encodedDataColumnPage.getByteArrayPage();
+} else {
+  return actualDataColumnPage.getByteArrayPage();
+}
+  }
+
+  /**
+   * Below method will be used to check whether page is local dictionary
+   * generated or not. This will be used for while enoding the the page
+   *
+   * @return
+   */
+  public boolean isLocalDictGeneratedPage() {
+return null != pageLevelDictionary;
+  }
+
+  /**
+   * Below method will be used to add column data to page
+   *
+   * @param rowId row number
+   * @param bytes actual data
+   */
+  @Override public void putBytes(int rowId, byte[] bytes) {
+if (null != pageLevelDictionary) {
+  try {
+actualDataColumnPage.putBytes(rowId, bytes);
+int dictionaryValue = 
pageLevelDictionary.getDictionaryValue(bytes);
+encodedDataColumnPage.putBytes(rowId, 
ByteUtil.toBytes(dictionaryValue));
+  } catch (DictionaryThresholdReachedException e) {
+LOGGER.error(e, "Local Dictionary threshold reached for the 
column: " + actualDataColumnPage
+.getColumnSpec().getFieldName());
+pageLevelDictionary = null;
+encodedDataColumnPage.freeMemory();
+encodedDataColumnPage = null;
+  }
+} else {

[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-23 Thread akashrn5
Github user akashrn5 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197604250
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala
 ---
@@ -96,8 +96,8 @@ object ExampleUtils {
 import spark.implicits._
 val sc = spark.sparkContext
 val df = sc.parallelize(1 to numRows, 2)
-.map(x => ("a", "b", x))
--- End diff --

remove the unnecessary changes


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-23 Thread akashrn5
Github user akashrn5 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197604461
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java 
---
@@ -104,19 +105,27 @@
 
page.setStatsCollector(KeyPageStatsCollector.newInstance(DataTypes.BYTE_ARRAY));
 dictDimensionPages[tmpNumDictDimIdx++] = page;
   } else {
+// will be encoded using string page
+LocalDictionaryGenerator localDictionaryGenerator =
+model.getColumnLocalDictGenMap().get(spec.getFieldName());
 if (DataTypes.VARCHAR == spec.getSchemaDataType()) {
-  page = ColumnPage.newPage(spec, DataTypes.VARCHAR, pageSize);
+  page = ColumnPage.newLocalDictPage(spec,
--- End diff --

if the `localDictionaryGenerator` is null, for VARCHAR also, it should not 
generate localDictPage, null check is missing i think


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-23 Thread akashrn5
Github user akashrn5 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197604074
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ---
@@ -482,7 +482,7 @@ public String getTableUniqueName() {
* @return
*/
   public boolean isLocalDictionaryEnabled() {
-return isLocalDictionaryEnabled;
--- End diff --

why it is always false


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-23 Thread akashrn5
Github user akashrn5 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197602851
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/blocklet/BlockletEncodedColumnPage.java
 ---
@@ -0,0 +1,187 @@
+/*
+ * 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.datastore.blocklet;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.page.FallbackColumnPageEncoder;
+import org.apache.carbondata.core.datastore.page.FallbackEncodedColumnPage;
+import 
org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
+import org.apache.carbondata.core.localdictionary.PageLevelDictionary;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.format.LocalDictionaryChunk;
+
+/**
+ * Maintains the list of encoded page of a column in a blocklet
+ * and encoded dictionary values only if column is encoded using local
+ * dictionary
+ * Handle the fallback if all the pages in blocklet are not
+ * encoded with local dictionary
+ */
+public class BlockletEncodedColumnPage {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+  
LogServiceFactory.getLogService(BlockletEncodedColumnPage.class.getName());
+
+  /**
+   * list of encoded page of a column in a blocklet
+   */
+  private List encodedColumnPageList;
+
+  /**
+   * fallback executor service
+   */
+  private ExecutorService fallbackExecutorService;
+
+  /**
+   * to check whether pages are local dictionary encoded or not
+   */
+  private boolean isLocalDictEncoded;
+
+  /**
+   * page level dictionary only when column is encoded with local 
dictionary
+   */
+  private PageLevelDictionary pageLevelDictionary;
+
+  /**
+   * fallback future task queue;
+   */
+  private ArrayDeque> 
fallbackFutureQueue;
+
+  BlockletEncodedColumnPage(ExecutorService fallbackExecutorService,
+  EncodedColumnPage encodedColumnPage) {
+this.encodedColumnPageList = new ArrayList<>();
+this.fallbackExecutorService = fallbackExecutorService;
+this.encodedColumnPageList.add(encodedColumnPage);
+// if dimension page is local dictionary enabled and encoded with 
local dictionary
+if (encodedColumnPage.isLocalDictionaryEnabled() && encodedColumnPage
+.isLocalDictGeneratedPage()) {
+  this.isLocalDictEncoded = true;
+  // get first page dictionary
+  this.pageLevelDictionary = encodedColumnPage.getPageDictionary();
+}
+  }
+
+  /**
+   * Below method will be used to add column page of a column
+   *
+   * @param encodedColumnPage
+   * encoded column page
+   * @throws ExecutionException
+   * failure in fallback
+   * @throws InterruptedException
+   * failure during fallback
+   */
+  void addEncodedColumnTable(EncodedColumnPage encodedColumnPage)
--- End diff --

change method name to `addEncodedColumnPage`


---


[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-23 Thread akashrn5
Github user akashrn5 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197603140
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/blocklet/BlockletEncodedColumnPage.java
 ---
@@ -0,0 +1,187 @@
+/*
+ * 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.datastore.blocklet;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.page.FallbackColumnPageEncoder;
+import org.apache.carbondata.core.datastore.page.FallbackEncodedColumnPage;
+import 
org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
+import org.apache.carbondata.core.localdictionary.PageLevelDictionary;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.format.LocalDictionaryChunk;
+
+/**
+ * Maintains the list of encoded page of a column in a blocklet
+ * and encoded dictionary values only if column is encoded using local
+ * dictionary
+ * Handle the fallback if all the pages in blocklet are not
+ * encoded with local dictionary
+ */
+public class BlockletEncodedColumnPage {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+  
LogServiceFactory.getLogService(BlockletEncodedColumnPage.class.getName());
+
+  /**
+   * list of encoded page of a column in a blocklet
+   */
+  private List encodedColumnPageList;
+
+  /**
+   * fallback executor service
+   */
+  private ExecutorService fallbackExecutorService;
+
+  /**
+   * to check whether pages are local dictionary encoded or not
+   */
+  private boolean isLocalDictEncoded;
+
+  /**
+   * page level dictionary only when column is encoded with local 
dictionary
+   */
+  private PageLevelDictionary pageLevelDictionary;
+
+  /**
+   * fallback future task queue;
+   */
+  private ArrayDeque> 
fallbackFutureQueue;
+
+  BlockletEncodedColumnPage(ExecutorService fallbackExecutorService,
+  EncodedColumnPage encodedColumnPage) {
+this.encodedColumnPageList = new ArrayList<>();
+this.fallbackExecutorService = fallbackExecutorService;
+this.encodedColumnPageList.add(encodedColumnPage);
+// if dimension page is local dictionary enabled and encoded with 
local dictionary
+if (encodedColumnPage.isLocalDictionaryEnabled() && encodedColumnPage
+.isLocalDictGeneratedPage()) {
+  this.isLocalDictEncoded = true;
+  // get first page dictionary
+  this.pageLevelDictionary = encodedColumnPage.getPageDictionary();
+}
+  }
+
+  /**
+   * Below method will be used to add column page of a column
+   *
+   * @param encodedColumnPage
+   * encoded column page
+   * @throws ExecutionException
+   * failure in fallback
+   * @throws InterruptedException
+   * failure during fallback
+   */
+  void addEncodedColumnTable(EncodedColumnPage encodedColumnPage)
+  throws ExecutionException, InterruptedException {
+// if local dictionary is false or column is encoded with local 
dictionary then
+// add a page
+if (!isLocalDictEncoded || 
encodedColumnPage.isLocalDictGeneratedPage()) {
+  this.encodedColumnPageList.add(encodedColumnPage);
+  // merge page level dictionary values
+  if (null != this.pageLevelDictionary) {
+
pageLevelDictionary.mergerDictionaryValues(encodedColumnPage.getPageDictionary());
+  }
+} else {
+  // if older pages where encoded with dictionary and new pages are 
with dictionary
 

[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-23 Thread akashrn5
Github user akashrn5 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2402#discussion_r197602985
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/blocklet/BlockletEncodedColumnPage.java
 ---
@@ -0,0 +1,187 @@
+/*
+ * 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.datastore.blocklet;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.page.FallbackColumnPageEncoder;
+import org.apache.carbondata.core.datastore.page.FallbackEncodedColumnPage;
+import 
org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
+import org.apache.carbondata.core.localdictionary.PageLevelDictionary;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.format.LocalDictionaryChunk;
+
+/**
+ * Maintains the list of encoded page of a column in a blocklet
+ * and encoded dictionary values only if column is encoded using local
+ * dictionary
+ * Handle the fallback if all the pages in blocklet are not
+ * encoded with local dictionary
+ */
+public class BlockletEncodedColumnPage {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+  
LogServiceFactory.getLogService(BlockletEncodedColumnPage.class.getName());
+
+  /**
+   * list of encoded page of a column in a blocklet
+   */
+  private List encodedColumnPageList;
+
+  /**
+   * fallback executor service
+   */
+  private ExecutorService fallbackExecutorService;
+
+  /**
+   * to check whether pages are local dictionary encoded or not
+   */
+  private boolean isLocalDictEncoded;
+
+  /**
+   * page level dictionary only when column is encoded with local 
dictionary
+   */
+  private PageLevelDictionary pageLevelDictionary;
+
+  /**
+   * fallback future task queue;
+   */
+  private ArrayDeque> 
fallbackFutureQueue;
+
+  BlockletEncodedColumnPage(ExecutorService fallbackExecutorService,
+  EncodedColumnPage encodedColumnPage) {
+this.encodedColumnPageList = new ArrayList<>();
+this.fallbackExecutorService = fallbackExecutorService;
+this.encodedColumnPageList.add(encodedColumnPage);
+// if dimension page is local dictionary enabled and encoded with 
local dictionary
+if (encodedColumnPage.isLocalDictionaryEnabled() && encodedColumnPage
+.isLocalDictGeneratedPage()) {
+  this.isLocalDictEncoded = true;
+  // get first page dictionary
+  this.pageLevelDictionary = encodedColumnPage.getPageDictionary();
+}
+  }
+
+  /**
+   * Below method will be used to add column page of a column
+   *
+   * @param encodedColumnPage
+   * encoded column page
+   * @throws ExecutionException
+   * failure in fallback
+   * @throws InterruptedException
+   * failure during fallback
+   */
+  void addEncodedColumnTable(EncodedColumnPage encodedColumnPage)
+  throws ExecutionException, InterruptedException {
+// if local dictionary is false or column is encoded with local 
dictionary then
+// add a page
+if (!isLocalDictEncoded || 
encodedColumnPage.isLocalDictGeneratedPage()) {
+  this.encodedColumnPageList.add(encodedColumnPage);
+  // merge page level dictionary values
+  if (null != this.pageLevelDictionary) {
+
pageLevelDictionary.mergerDictionaryValues(encodedColumnPage.getPageDictionary());
+  }
+} else {
+  // if older pages where encoded with dictionary and new pages are 
with dictionary
 

[GitHub] carbondata pull request #2402: [CARBONDATA-2587][CARBONDATA-2588] Local Dict...

2018-06-22 Thread kumarvishal09
GitHub user kumarvishal09 opened a pull request:

https://github.com/apache/carbondata/pull/2402

[CARBONDATA-2587][CARBONDATA-2588] Local Dictionary Data Loading support

Be sure to do all of the following checklist to help us incorporate 
your contribution quickly and easily:

 - [ ] Any interfaces changed?
 
 - [ ] Any backward compatibility impacted?
 
 - [ ] Document update required?

 - [ ] Testing done
Please provide details on 
- Whether new unit test cases have been added or why no new tests 
are required?
- How it is tested? Please attach test report.
- Is it a performance related change? Please attach the performance 
test report.
- Any additional information to help reviewers in testing this 
change.
   
 - [ ] For large changes, please consider breaking it into sub-tasks under 
an umbrella JIRA. 



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/kumarvishal09/incubator-carbondata 
branch_localdic

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/carbondata/pull/2402.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #2402


commit 067527cce26e60eb2bbadf8536dd45bc90a2e680
Author: kumarvishal09 
Date:   2018-06-04T10:11:50Z

local dictionary code




---