carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ravipes...@apache.org
Subject [2/9] incubator-carbondata git commit: clean up core
Date Wed, 04 Jan 2017 13:34:56 GMT
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
index 4eed732..78d3185 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
@@ -121,15 +121,15 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToDeleteFolderForValidPath()
-      throws CarbonUtilException, InterruptedException {
+      throws InterruptedException, IOException {
     File testDir = new File("../core/src/test/resources/testDir");
     testDir.mkdirs();
     CarbonUtil.deleteFoldersAndFiles(testDir);
     assertTrue(!testDir.isDirectory());
   }
 
-  @Test(expected = CarbonUtilException.class) public void testToDeleteFolderWithIOException()
-      throws CarbonUtilException, InterruptedException {
+  @Test(expected = IOException.class) public void testToDeleteFolderWithIOException()
+      throws InterruptedException, IOException {
     File testDir = new File("../core/src/test/resources/testDir");
     new MockUp<UserGroupInformation>() {
       @SuppressWarnings("unused") @Mock public UserGroupInformation getLoginUser()
@@ -140,9 +140,9 @@ public class CarbonUtilTest {
     CarbonUtil.deleteFoldersAndFiles(testDir);
   }
 
-  @Test(expected = CarbonUtilException.class)
+  @Test(expected = InterruptedException.class)
   public void testToDeleteFolderWithInterruptedException()
-      throws CarbonUtilException, InterruptedException {
+      throws InterruptedException, IOException {
     File testDir = new File("../core/src/test/resources/testDir");
     new MockUp<UserGroupInformation>() {
       @SuppressWarnings("unused") @Mock public UserGroupInformation getLoginUser()
@@ -154,7 +154,7 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToDeleteFileForValidPath()
-      throws CarbonUtilException, InterruptedException {
+      throws InterruptedException, IOException {
     File testDir = new File("../core/src/test/resources/testDir/testFile.csv");
     testDir.mkdirs();
     CarbonUtil.deleteFoldersAndFiles(testDir);
@@ -162,7 +162,7 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToDeleteFoldersAndFilesForValidFolder()
-      throws CarbonUtilException, InterruptedException {
+      throws InterruptedException, IOException {
     String folderPath = "../core/src/test/resources/testDir/carbonDir";
     new File(folderPath).mkdirs();
     LocalCarbonFile testDir = new LocalCarbonFile(folderPath);
@@ -170,8 +170,8 @@ public class CarbonUtilTest {
     assertTrue(!testDir.exists());
   }
 
-  @Test(expected = CarbonUtilException.class) public void testToDeleteFoldersAndFilesWithIOException()
-      throws CarbonUtilException, InterruptedException {
+  @Test(expected = IOException.class) public void testToDeleteFoldersAndFilesWithIOException()
+      throws InterruptedException, IOException {
     LocalCarbonFile testDir = new LocalCarbonFile("../core/src/test/resources/testDir/carbonDir");
     new MockUp<UserGroupInformation>() {
       @SuppressWarnings("unused") @Mock public UserGroupInformation getLoginUser()
@@ -182,8 +182,8 @@ public class CarbonUtilTest {
     CarbonUtil.deleteFoldersAndFiles(testDir);
   }
 
-  @Test(expected = CarbonUtilException.class) public void testToDeleteFoldersAndFilesWithInterruptedException()
-      throws CarbonUtilException, InterruptedException {
+  @Test(expected = InterruptedException.class) public void testToDeleteFoldersAndFilesWithInterruptedException()
+      throws InterruptedException, IOException {
     LocalCarbonFile testDir = new LocalCarbonFile("../core/src/test/resources/testDir/carbonDir");
     new MockUp<UserGroupInformation>() {
       @SuppressWarnings("unused") @Mock public UserGroupInformation getLoginUser()
@@ -195,7 +195,7 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToDeleteFoldersAndFilesForValidCarbonFile()
-      throws CarbonUtilException, InterruptedException {
+      throws InterruptedException, IOException {
     LocalCarbonFile testDir =
         new LocalCarbonFile("../core/src/test/resources/testDir/testCarbonFile");
     testDir.createNewFile();
@@ -203,7 +203,7 @@ public class CarbonUtilTest {
     assertTrue(!testDir.exists());
   }
 
-  @Test public void testToGetBadLogPath() throws CarbonUtilException, InterruptedException {
+  @Test public void testToGetBadLogPath() throws InterruptedException {
     new MockUp<CarbonProperties>() {
       @SuppressWarnings("unused") @Mock public String getProperty(String key) {
         return "../unibi-solutions/system/carbon/badRecords";
@@ -214,16 +214,16 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToDeleteFoldersAndFilesForCarbonFileSilently()
-      throws CarbonUtilException, InterruptedException {
+      throws IOException, InterruptedException {
     LocalCarbonFile testDir = new LocalCarbonFile("../core/src/test/resources/testDir");
     testDir.createNewFile();
     CarbonUtil.deleteFoldersAndFilesSilent(testDir);
     assertTrue(!testDir.exists());
   }
 
-  @Test(expected = CarbonUtilException.class)
+  @Test(expected = IOException.class)
   public void testToDeleteFoldersAndFilesSintlyWithIOException()
-      throws CarbonUtilException, IOException {
+      throws IOException, InterruptedException {
     new MockUp<UserGroupInformation>() {
       @SuppressWarnings("unused") @Mock public UserGroupInformation getLoginUser()
           throws IOException {
@@ -235,9 +235,9 @@ public class CarbonUtilTest {
     CarbonUtil.deleteFoldersAndFilesSilent(testDir);
   }
 
-  @Test(expected = CarbonUtilException.class)
+  @Test(expected = InterruptedException.class)
   public void testToDeleteFoldersAndFilesSintlyWithInterruptedException()
-      throws CarbonUtilException, IOException {
+      throws IOException, InterruptedException {
     new MockUp<UserGroupInformation>() {
       @SuppressWarnings("unused") @Mock public UserGroupInformation getLoginUser()
           throws InterruptedException {
@@ -249,7 +249,7 @@ public class CarbonUtilTest {
     CarbonUtil.deleteFoldersAndFilesSilent(testDir);
   }
 
-  @Test public void testToDeleteFiles() throws IOException, CarbonUtilException {
+  @Test public void testToDeleteFiles() throws IOException {
     String baseDirectory = "../core/src/test/resources/";
     File file1 = new File(baseDirectory + "File1.txt");
     File file2 = new File(baseDirectory + "File2.txt");
@@ -341,7 +341,8 @@ public class CarbonUtilTest {
         .exists());
   }
 
-  @Test public void testToGetCardinalityFromLevelMetadataFile() throws CarbonUtilException {
+  @Test public void testToGetCardinalityFromLevelMetadataFile()
+      throws IOException, InterruptedException {
     int[] cardinality = CarbonUtil.getCardinalityFromLevelMetadataFile(
         "../core/src/test/resources/testDatabase/levelmetadata_testTable.metadata");
     int[] expectedCardinality = { 10, 20, 30, 40 };
@@ -351,7 +352,7 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToGetCardinalityFromLevelMetadataFileForInvalidPath()
-      throws CarbonUtilException {
+      throws IOException, InterruptedException {
     int[] cardinality = CarbonUtil.getCardinalityFromLevelMetadataFile("");
     assertEquals(cardinality, null);
   }
@@ -550,7 +551,7 @@ public class CarbonUtilTest {
     }
   }
 
-  @Test public void testToReadMetadatFile() throws CarbonUtilException {
+  @Test public void testToReadMetadatFile() throws IOException {
     new MockUp<DataFileFooterConverter>() {
       @SuppressWarnings("unused") @Mock
       public DataFileFooter readDataFileFooter(TableBlockInfo info) {
@@ -564,7 +565,8 @@ public class CarbonUtilTest {
     assertEquals(CarbonUtil.readMetadatFile(info).getVersionId().number(), 1);
   }
 
-  @Test(expected = CarbonUtilException.class) public void testToReadMetadatFileWithException()
+  @Test(expected = IOException.class)
+  public void testToReadMetadatFileWithException()
       throws Exception {
 	TableBlockInfo info = new TableBlockInfo("file:/", 1, "0", new String[0], 1, ColumnarFormatVersion.V1);
     CarbonUtil.readMetadatFile(info);
@@ -632,7 +634,8 @@ public class CarbonUtilTest {
     file.deleteOnExit();
   }
 
-  @Test public void testToReadHeaderWithFileNotFoundException() throws IOException {
+  @Test(expected = IOException.class)
+  public void testToReadHeaderWithFileNotFoundException() throws IOException {
     new MockUp<FileFactory>() {
       @SuppressWarnings("unused") @Mock
       public DataInputStream getDataInputStream(String path, FileFactory.FileType fileType)
@@ -644,7 +647,8 @@ public class CarbonUtilTest {
     assertEquals(null, result);
   }
 
-  @Test public void testToReadHeaderWithIOException() throws IOException {
+  @Test(expected = IOException.class)
+  public void testToReadHeaderWithIOException() throws IOException {
     new MockUp<FileFactory>() {
       @SuppressWarnings("unused") @Mock
       public DataInputStream getDataInputStream(String path, FileFactory.FileType fileType)
@@ -688,7 +692,7 @@ public class CarbonUtilTest {
 
     List<Encoding> encodingList = new ArrayList<>();
     encodingList.add(Encoding.DELTA);
-    dataChunk.setEncoderList(encodingList);
+    dataChunk.setEncodingList(encodingList);
 
     List<ValueEncoderMeta> valueEncoderMetas = new ArrayList<>();
     ValueEncoderMeta valueEncoderMeta = new ValueEncoderMeta();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/core/src/test/java/org/apache/carbondata/core/writer/CarbonFooterWriterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/CarbonFooterWriterTest.java b/core/src/test/java/org/apache/carbondata/core/writer/CarbonFooterWriterTest.java
index 20a0790..432dfbf 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/CarbonFooterWriterTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/CarbonFooterWriterTest.java
@@ -122,8 +122,8 @@ public class CarbonFooterWriterTest extends TestCase{
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
    encodeList.add(Encoding.DICTIONARY);
    dimColumn.setEncodingList(encodeList);
-   dimColumn.setNumberOfChild(0);;
-   return dimColumn;
+   dimColumn.setNumberOfChild(0);
+    return dimColumn;
  }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
index e0c444b..246fe1d 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
@@ -24,7 +24,6 @@ import java.util.List;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryChunksWrapper;
 import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
-import org.apache.carbondata.core.util.CarbonUtilException;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -47,10 +46,8 @@ public class CarbonDictionarySortInfoPreparatorTest {
 
   /**
    * Tests the getDictionarySortInfo method
-   *
-   * @throws CarbonUtilException
    */
-  @Test public void testGetDictionarySortInfo() throws CarbonUtilException {
+  @Test public void testGetDictionarySortInfo() {
 
     List<String> newDistinctValues = new ArrayList<>();
     newDistinctValues.add("abc");
@@ -83,10 +80,8 @@ public class CarbonDictionarySortInfoPreparatorTest {
 
   /**
    * Tests getDictionarySortInfo when dictionary is null
-   *
-   * @throws CarbonUtilException
    */
-  @Test public void testGetDictionarySortInfoDictionaryNullCase() throws CarbonUtilException {
+  @Test public void testGetDictionarySortInfoDictionaryNullCase() {
 
     List<String> newDistinctValues = new ArrayList<>();
     newDistinctValues.add("abc");

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/core/src/test/java/org/apache/carbondata/scan/collector/impl/RawBasedResultCollectorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/scan/collector/impl/RawBasedResultCollectorTest.java b/core/src/test/java/org/apache/carbondata/scan/collector/impl/RawBasedResultCollectorTest.java
index 78b5916..0d75942 100644
--- a/core/src/test/java/org/apache/carbondata/scan/collector/impl/RawBasedResultCollectorTest.java
+++ b/core/src/test/java/org/apache/carbondata/scan/collector/impl/RawBasedResultCollectorTest.java
@@ -69,7 +69,6 @@ public class RawBasedResultCollectorTest {
     KeyStructureInfo keyStructureInfo = new KeyStructureInfo();
     keyStructureInfo.setKeyGenerator(keyGenerator);
     keyStructureInfo.setMaxKey(new byte[] { 1, 2 });
-    keyStructureInfo.setMaskedBytes(new int[] { 1, 2 });
     keyStructureInfo.setMaskByteRanges(new int[] { 1, 2 });
     blockExecutionInfo.setKeyStructureInfo(keyStructureInfo);
     AggregatorInfo aggregatorInfo = new AggregatorInfo();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/core/src/test/java/org/apache/carbondata/scan/filter/FilterUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/scan/filter/FilterUtilTest.java b/core/src/test/java/org/apache/carbondata/scan/filter/FilterUtilTest.java
index 53fdc09..c93713a 100644
--- a/core/src/test/java/org/apache/carbondata/scan/filter/FilterUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/scan/filter/FilterUtilTest.java
@@ -38,7 +38,6 @@ import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSche
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
-import org.apache.carbondata.scan.executor.exception.QueryExecutionException;
 import org.apache.carbondata.scan.expression.ColumnExpression;
 import org.apache.carbondata.scan.expression.Expression;
 import org.apache.carbondata.scan.expression.LiteralExpression;
@@ -334,67 +333,6 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
         defaultSurrogate) instanceof DimColumnFilterInfo);
   }
 
-  @Test public void testGetFilterList() throws Exception {
-    Expression expression = new ColumnExpression("IMEI", DataType.STRING);
-    ColumnExpression columnExpression = new ColumnExpression("IMEI", DataType.STRING);
-    int ordinal = 1;
-    boolean isIncludeFilter = true;
-    CarbonTableIdentifier carbonTableIdentifier =
-        new CarbonTableIdentifier("databaseName", "tableName", "tableID");
-    AbsoluteTableIdentifier absoluteTableIdentifier =
-        new AbsoluteTableIdentifier(this.carbonStorePath, carbonTableIdentifier);
-    final Map<String, String> columnProperties = new HashMap<>();
-    columnProperties.put("1", "column1");
-    columnProperties.put("2", "column2");
-    final CarbonTableIdentifier carbonTableIdentifier1 =
-        new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    int keyOrdinal = 1;
-    int columnGroupOrdinal = 1;
-    int complexTypeOrdinal = 1;
-    final CarbonDimension carbonDimension =
-        new CarbonDimension(columnSchema, ordinal, keyOrdinal, columnGroupOrdinal,
-            complexTypeOrdinal);
-
-    new MockUp<AbsoluteTableIdentifier>() {
-      @Mock public CarbonTableIdentifier getCarbonTableIdentifier() {
-        return carbonTableIdentifier1;
-      }
-    };
-
-    final Map<String, String> columnProperties1 = new HashMap<>();
-    columnProperties1.put("1", "column1");
-    columnProperties1.put("2", "column2");
-
-    new MockUp<CarbonColumn>() {
-      @Mock public ColumnIdentifier getColumnIdentifier() {
-        ColumnIdentifier columnIdentifier =
-            new ColumnIdentifier("1", columnProperties, DataType.STRING);
-        return columnIdentifier;
-      }
-    };
-
-    new MockUp<CarbonColumn>() {
-      @Mock public DataType getDataType() {
-        return DataType.STRING;
-      }
-    };
-
-    new MockUp<ColumnExpression>() {
-      @Mock public CarbonDimension getDimension() {
-        return carbonDimension;
-      }
-    };
-
-    final CarbonColumn carbonColumn = new CarbonColumn(columnSchema, ordinal, -1);
-    new MockUp<ColumnExpression>() {
-      @Mock public CarbonColumn getCarbonColumn() {
-        return carbonColumn;
-      }
-    };
-    assertTrue(FilterUtil.getFilterList(absoluteTableIdentifier, expression, columnExpression,
-        isIncludeFilter) instanceof DimColumnFilterInfo);
-  }
-
   @Test public void testCheckIfDataTypeNotTimeStamp() {
     Expression expression = new ColumnExpression("test", DataType.STRING);
     boolean result = FilterUtil.checkIfDataTypeNotTimeStamp(expression);
@@ -437,8 +375,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     List<String> evaluateResultListFinal = new ArrayList<>();
     evaluateResultListFinal.add("test1");
     evaluateResultListFinal.add("test2");
-    assertTrue(FilterUtil.getNoDictionaryValKeyMemberForFilter(absoluteTableIdentifier, expression,
-        evaluateResultListFinal, isIncludeFilter) instanceof DimColumnFilterInfo);
+    assertTrue(FilterUtil.getNoDictionaryValKeyMemberForFilter(evaluateResultListFinal, isIncludeFilter) instanceof DimColumnFilterInfo);
   }
 
   @Test public void testPrepareDefaultStartIndexKey() throws KeyGenException {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java b/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java
index 3ff131b..d1396e5 100644
--- a/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java
+++ b/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.carbondata.scanner.impl;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.BitSet;
 import java.util.HashMap;
@@ -41,7 +42,6 @@ import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsRecorder
 import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsRecorderImpl;
 import org.apache.carbondata.core.datastorage.store.FileHolder;
 import org.apache.carbondata.core.datastorage.store.impl.DFSFileHolderImpl;
-import org.apache.carbondata.scan.executor.exception.QueryExecutionException;
 import org.apache.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.scan.executor.infos.KeyStructureInfo;
 import org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
@@ -103,7 +103,8 @@ public class FilterScannerTest {
     bTreeBuilderInfo = new BTreeBuilderInfo(Arrays.asList(dataFileFooter), new int[] { 1 });
   }
 
-  @Test public void testToScanBlockletWithEmptyBitSet() throws QueryExecutionException {
+  @Test public void testToScanBlockletWithEmptyBitSet()
+      throws IOException, FilterUnsupportedException {
     new MockUp<AndFilterExecuterImpl>() {
       @SuppressWarnings("unused") @Mock
       public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
@@ -117,7 +118,8 @@ public class FilterScannerTest {
     assertEquals(0, abstractScannedResult.numberOfOutputRows());
   }
 
-  @Test public void testToScanBlockletWithNonEmptyBitSet() throws QueryExecutionException {
+  @Test public void testToScanBlockletWithNonEmptyBitSet()
+      throws IOException, FilterUnsupportedException {
     new MockUp<AndFilterExecuterImpl>() {
       @SuppressWarnings("unused") @Mock
       public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
@@ -173,8 +175,8 @@ public class FilterScannerTest {
     assertEquals(2, abstractScannedResult.numberOfOutputRows());
   }
 
-  @Test(expected = QueryExecutionException.class) public void testToScanBlockletWithException()
-      throws QueryExecutionException {
+  @Test(expected = FilterUnsupportedException.class) public void testToScanBlockletWithException()
+      throws IOException, FilterUnsupportedException {
     new MockUp<AndFilterExecuterImpl>() {
       @SuppressWarnings("unused") @Mock
       public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheAccessClient.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheAccessClient.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheAccessClient.java
index 2760035..984930d 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheAccessClient.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheAccessClient.java
@@ -18,13 +18,12 @@
  */
 package org.apache.carbondata.hadoop;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.util.CarbonUtilException;
 
 /**
  * CacheClient : Class used to request the segments cache
@@ -35,11 +34,6 @@ public class CacheAccessClient<K, V> {
    */
   private List<K> segmentList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
-  /**
-   * absolute table identifier
-   */
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
-
   private Cache<K, V> cache;
 
   public CacheAccessClient(Cache cache) {
@@ -67,9 +61,9 @@ public class CacheAccessClient<K, V> {
    *
    * @param key
    * @return
-   * @throws CarbonUtilException in case memory is not sufficient to load data into memory
+   * @throws IOException in case memory is not sufficient to load data into memory
    */
-  public V get(K key) throws CarbonUtilException {
+  public V get(K key) throws IOException {
     V value = cache.get(key);
     if (value != null) {
       segmentList.add(key);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
index 7e9dc7a..1e4f821 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
@@ -38,7 +38,6 @@ import org.apache.carbondata.core.carbon.datastore.block.BlockletInfos;
 import org.apache.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.carbon.datastore.block.SegmentTaskIndexWrapper;
 import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.carbon.datastore.exception.IndexBuilderException;
 import org.apache.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
 import org.apache.carbondata.core.carbon.datastore.impl.btree.BlockBTreeLeafNode;
 import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable;
@@ -51,14 +50,12 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.CarbonUtilException;
 import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
 import org.apache.carbondata.hadoop.readsupport.impl.DictionaryDecodedReadSupportImpl;
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
 import org.apache.carbondata.hadoop.util.ObjectSerializationUtil;
 import org.apache.carbondata.hadoop.util.SchemaReader;
 import org.apache.carbondata.lcm.status.SegmentStatusManager;
-import org.apache.carbondata.scan.executor.exception.QueryExecutionException;
 import org.apache.carbondata.scan.expression.Expression;
 import org.apache.carbondata.scan.filter.FilterExpressionProcessor;
 import org.apache.carbondata.scan.filter.FilterUtil;
@@ -240,17 +237,9 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
     CarbonTable carbonTable = getCarbonTable(job.getConfiguration());
     CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
     FilterResolverIntf filterInterface = CarbonInputFormatUtil.resolveFilter(filter, identifier);
-    List<InputSplit> splits;
-    try {
-      // do block filtering and get split
-      splits = getSplits(job, filterInterface, cacheClient);
-    } catch (IndexBuilderException | CarbonUtilException e) {
-      throw new IOException(e);
-    }
-    finally {
-      cacheClient.close();
-      cacheClient = null;
-    }
+    // do block filtering and get split
+    List<InputSplit> splits = getSplits(job, filterInterface, cacheClient);
+    cacheClient.close();
     // pass the invalid segment to task side in order to remove index entry in task side
     if (invalidSegments.size() > 0) {
       for (InputSplit split : splits) {
@@ -286,7 +275,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
    * @throws IOException
    */
   private List<InputSplit> getSplits(JobContext job, FilterResolverIntf filterResolver,
-      CacheClient cacheClient) throws IOException, IndexBuilderException, CarbonUtilException {
+      CacheClient cacheClient) throws IOException {
 
     List<InputSplit> result = new LinkedList<InputSplit>();
 
@@ -330,8 +319,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
   private List<DataRefNode> getDataBlocksOfSegment(JobContext job,
       FilterExpressionProcessor filterExpressionProcessor,
       AbsoluteTableIdentifier absoluteTableIdentifier, FilterResolverIntf resolver,
-      String segmentId, CacheClient cacheClient)
-      throws IndexBuilderException, IOException, CarbonUtilException {
+      String segmentId, CacheClient cacheClient) throws IOException {
     QueryStatisticsRecorder recorder = CarbonTimeStatisticsFactory.createDriverRecorder();
     QueryStatistic statistic = new QueryStatistic();
     Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> segmentIndexMap =
@@ -341,20 +329,18 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
 
     // build result
     for (AbstractIndex abstractIndex : segmentIndexMap.values()) {
-
-      List<DataRefNode> filterredBlocks = null;
+      List<DataRefNode> filterredBlocks;
       // if no filter is given get all blocks from Btree Index
       if (null == resolver) {
         filterredBlocks = getDataBlocksOfIndex(abstractIndex);
       } else {
         // apply filter and get matching blocks
-        try {
-          filterredBlocks = filterExpressionProcessor
-              .getFilterredBlocks(abstractIndex.getDataRefNode(), resolver, abstractIndex,
-                  absoluteTableIdentifier);
-        } catch (QueryExecutionException e) {
-          throw new IndexBuilderException(e.getMessage());
-        }
+        filterredBlocks = filterExpressionProcessor.getFilterredBlocks(
+            abstractIndex.getDataRefNode(),
+            resolver,
+            abstractIndex,
+            absoluteTableIdentifier
+        );
       }
       resultFilterredBlocks.addAll(filterredBlocks);
     }
@@ -401,11 +387,10 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
    * @param segmentId
    * @return
    * @throws IOException
-   * @throws IndexBuilderException
    */
   private Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> getSegmentAbstractIndexs(
       JobContext job, AbsoluteTableIdentifier absoluteTableIdentifier, String segmentId,
-      CacheClient cacheClient) throws IOException, IndexBuilderException, CarbonUtilException {
+      CacheClient cacheClient) throws IOException {
     Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> segmentIndexMap = null;
     TableSegmentUniqueIdentifier tableSegmentUniqueIdentifier =
         new TableSegmentUniqueIdentifier(absoluteTableIdentifier, segmentId);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
index 9d8b136..16f0fbb 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
@@ -37,7 +37,6 @@ import org.apache.carbondata.core.carbon.datastore.block.BlockletInfos;
 import org.apache.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.carbon.datastore.block.SegmentTaskIndexWrapper;
 import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.carbon.datastore.exception.IndexBuilderException;
 import org.apache.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
 import org.apache.carbondata.core.carbon.datastore.impl.btree.BlockBTreeLeafNode;
 import org.apache.carbondata.core.carbon.querystatistics.QueryStatistic;
@@ -45,13 +44,11 @@ import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsConstant
 import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsRecorder;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.core.util.CarbonUtilException;
 import org.apache.carbondata.hadoop.CacheClient;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
 import org.apache.carbondata.hadoop.internal.index.Block;
 import org.apache.carbondata.hadoop.internal.index.Index;
 import org.apache.carbondata.hadoop.internal.segment.Segment;
-import org.apache.carbondata.scan.executor.exception.QueryExecutionException;
 import org.apache.carbondata.scan.filter.FilterExpressionProcessor;
 import org.apache.carbondata.scan.filter.FilterUtil;
 import org.apache.carbondata.scan.filter.resolver.FilterResolverIntf;
@@ -87,12 +84,8 @@ class InMemoryBTreeIndex implements Index {
     AbsoluteTableIdentifier identifier = null;
 
     //for this segment fetch blocks matching filter in BTree
-    List<DataRefNode> dataRefNodes = null;
-    try {
-      dataRefNodes = getDataBlocksOfSegment(job, filterExpressionProcessor, identifier, filter);
-    } catch (IndexBuilderException e) {
-      throw new IOException(e.getMessage());
-    }
+    List<DataRefNode> dataRefNodes =
+        getDataBlocksOfSegment(job, filterExpressionProcessor, identifier, filter);
     for (DataRefNode dataRefNode : dataRefNodes) {
       BlockBTreeLeafNode leafNode = (BlockBTreeLeafNode) dataRefNode;
       TableBlockInfo tableBlockInfo = leafNode.getTableBlockInfo();
@@ -105,8 +98,7 @@ class InMemoryBTreeIndex implements Index {
   }
 
   private Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> getSegmentAbstractIndexs(
-      JobContext job, AbsoluteTableIdentifier identifier)
-      throws IOException, IndexBuilderException {
+      JobContext job, AbsoluteTableIdentifier identifier) throws IOException {
     Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> segmentIndexMap = null;
     CacheClient cacheClient = new CacheClient(identifier.getStorePath());
     TableSegmentUniqueIdentifier segmentUniqueIdentifier =
@@ -128,11 +120,7 @@ class InMemoryBTreeIndex implements Index {
         segmentTaskIndexWrapper = cacheClient.getSegmentAccessClient().get(segmentUniqueIdentifier);
         segmentIndexMap = segmentTaskIndexWrapper.getTaskIdToTableSegmentMap();
       }
-    }
-    catch (CarbonUtilException e) {
-      throw new IndexBuilderException(e.getMessage(), e);
-    }
-    finally {
+    } finally {
       cacheClient.close();
     }
     return segmentIndexMap;
@@ -166,7 +154,7 @@ class InMemoryBTreeIndex implements Index {
    */
   private List<DataRefNode> getDataBlocksOfSegment(JobContext job,
       FilterExpressionProcessor filterExpressionProcessor, AbsoluteTableIdentifier identifier,
-      FilterResolverIntf resolver) throws IndexBuilderException, IOException {
+      FilterResolverIntf resolver) throws IOException {
 
     QueryStatisticsRecorder recorder = CarbonTimeStatisticsFactory.createDriverRecorder();
     QueryStatistic statistic = new QueryStatistic();
@@ -184,13 +172,12 @@ class InMemoryBTreeIndex implements Index {
         filterredBlocks = getDataBlocksOfIndex(abstractIndex);
       } else {
         // apply filter and get matching blocks
-        try {
-          filterredBlocks =
-              filterExpressionProcessor.getFilterredBlocks(abstractIndex.getDataRefNode(),
-                  resolver, abstractIndex, identifier);
-        } catch (QueryExecutionException e) {
-          throw new IndexBuilderException(e.getMessage());
-        }
+        filterredBlocks = filterExpressionProcessor.getFilterredBlocks(
+            abstractIndex.getDataRefNode(),
+            resolver,
+            abstractIndex,
+            identifier
+        );
       }
       resultFilterredBlocks.addAll(filterredBlocks);
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java
index 1b7f577..87da719 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java
@@ -18,6 +18,8 @@
  */
 package org.apache.carbondata.hadoop.readsupport;
 
+import java.io.IOException;
+
 import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
 
@@ -31,10 +33,10 @@ public interface CarbonReadSupport<T> {
    *
    * @param carbonColumns
    */
-  public void initialize(CarbonColumn[] carbonColumns,
-      AbsoluteTableIdentifier absoluteTableIdentifier);
+  void initialize(CarbonColumn[] carbonColumns,
+      AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException;
 
-  public T readRow(Object[] data);
+  T readRow(Object[] data);
 
   /**
    * This method will be used to clear the dictionary cache and update access count for each

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
index 5eda4d8..3626163 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
@@ -18,6 +18,8 @@
  */
 package org.apache.carbondata.hadoop.readsupport.impl;
 
+import java.io.IOException;
+
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.CacheProvider;
 import org.apache.carbondata.core.cache.CacheType;
@@ -28,7 +30,6 @@ import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
 import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.CarbonUtilException;
 import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
 
 /**
@@ -52,7 +53,7 @@ public abstract class AbstractDictionaryDecodedReadSupport<T> implements CarbonR
    * @param absoluteTableIdentifier
    */
   @Override public void initialize(CarbonColumn[] carbonColumns,
-      AbsoluteTableIdentifier absoluteTableIdentifier) {
+      AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException {
     this.carbonColumns = carbonColumns;
     dictionaries = new Dictionary[carbonColumns.length];
     dataTypes = new DataType[carbonColumns.length];
@@ -62,14 +63,10 @@ public abstract class AbstractDictionaryDecodedReadSupport<T> implements CarbonR
         CacheProvider cacheProvider = CacheProvider.getInstance();
         Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache = cacheProvider
             .createCache(CacheType.FORWARD_DICTIONARY, absoluteTableIdentifier.getStorePath());
-        try {
-          dataTypes[i] = carbonColumns[i].getDataType();
-          dictionaries[i] = forwardDictionaryCache.get(new DictionaryColumnUniqueIdentifier(
-              absoluteTableIdentifier.getCarbonTableIdentifier(),
-              carbonColumns[i].getColumnIdentifier(), dataTypes[i]));
-        } catch (CarbonUtilException e) {
-          throw new RuntimeException(e);
-        }
+        dataTypes[i] = carbonColumns[i].getDataType();
+        dictionaries[i] = forwardDictionaryCache.get(new DictionaryColumnUniqueIdentifier(
+            absoluteTableIdentifier.getCarbonTableIdentifier(),
+            carbonColumns[i].getColumnIdentifier(), dataTypes[i]));
       } else {
         dataTypes[i] = carbonColumns[i].getDataType();
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
index 2a70cca..9100268 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -72,7 +72,6 @@ public class CarbonInputFormatUtil {
       }
     }
 
-    plan.setLimit(-1);
     plan.setQueryId(System.nanoTime() + "");
     return plan;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
index 4519e90..e983f86 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
@@ -70,7 +70,6 @@ import org.apache.carbondata.processing.dataprocessor.DataProcessTaskStatus;
 import org.apache.carbondata.processing.dataprocessor.IDataProcessStatus;
 import org.apache.carbondata.processing.graphgenerator.GraphGenerator;
 import org.apache.carbondata.processing.graphgenerator.GraphGeneratorException;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
 import java.io.*;
 import java.nio.charset.Charset;
@@ -374,8 +373,7 @@ public class StoreCreator {
 
     DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus);
     graphExecuter
-        .executeGraph(graphPath, new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN),
-            info, "0", loadModel.getSchema());
+        .executeGraph(graphPath, info, loadModel.getSchema());
     //    LoadMetadataDetails[] loadDetails =
     //        CarbonUtil.readLoadMetadata(loadModel.schema.getCarbonTable().getMetaDataFilepath());
     writeLoadMetadata(loadModel.schema, loadModel.getTableName(), loadModel.getTableName(),

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
index 419e833..7a88b63 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
@@ -63,7 +63,6 @@ import org.apache.carbondata.core.datastorage.store.impl.FileFactory.FileType;
 import org.apache.carbondata.core.load.LoadMetadataDetails;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.CarbonUtilException;
 import org.apache.carbondata.lcm.fileoperations.AtomicFileOperations;
 import org.apache.carbondata.lcm.fileoperations.AtomicFileOperationsImpl;
 import org.apache.carbondata.lcm.fileoperations.FileWriteOperation;
@@ -174,8 +173,7 @@ public final class CarbonLoaderUtil {
 
     DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus);
     graphExecuter
-        .executeGraph(graphPath, new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN),
-            info, loadModel.getPartitionId(), loadModel.getCarbonDataLoadSchema());
+        .executeGraph(graphPath, info, loadModel.getCarbonDataLoadSchema());
   }
 
   public static List<String> addNewSliceNameToList(String newSlice, List<String> activeSlices) {
@@ -248,9 +246,7 @@ public final class CarbonLoaderUtil {
         CarbonFile carbonFile = FileFactory.getCarbonFile(path, fileType);
         CarbonUtil.deleteFoldersAndFiles(carbonFile);
       }
-    } catch (IOException e) {
-      LOGGER.error("Unable to delete the given path :: " + e.getMessage());
-    } catch (CarbonUtilException e) {
+    } catch (IOException | InterruptedException e) {
       LOGGER.error("Unable to delete the given path :: " + e.getMessage());
     }
   }
@@ -313,7 +309,7 @@ public final class CarbonLoaderUtil {
     try {
       CarbonUtil.deleteFoldersAndFiles(new File(localStoreLocation).getParentFile());
       LOGGER.info("Deleted the local store location" + localStoreLocation);
-    } catch (CarbonUtilException e) {
+    } catch (IOException | InterruptedException e) {
       LOGGER.error(e, "Failed to delete local data load folder location");
     }
 
@@ -458,7 +454,7 @@ public final class CarbonLoaderUtil {
   }
 
   public static Dictionary getDictionary(DictionaryColumnUniqueIdentifier columnIdentifier,
-      String carbonStorePath) throws CarbonUtilException {
+      String carbonStorePath) throws IOException {
     Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictCache =
         CacheProvider.getInstance().createCache(CacheType.REVERSE_DICTIONARY, carbonStorePath);
     return dictCache.get(columnIdentifier);
@@ -466,7 +462,7 @@ public final class CarbonLoaderUtil {
 
   public static Dictionary getDictionary(CarbonTableIdentifier tableIdentifier,
       ColumnIdentifier columnIdentifier, String carbonStorePath, DataType dataType)
-      throws CarbonUtilException {
+      throws IOException {
     return getDictionary(
         new DictionaryColumnUniqueIdentifier(tableIdentifier, columnIdentifier, dataType),
         carbonStorePath);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionExecutor.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionExecutor.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionExecutor.java
index ef3d2e4..e06a223 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionExecutor.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionExecutor.java
@@ -18,6 +18,7 @@
  */
 package org.apache.carbondata.spark.merger;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -81,7 +82,7 @@ public class CarbonCompactionExecutor {
    *
    * @return List of Carbon iterators
    */
-  public List<RawResultIterator> processTableBlocks() throws QueryExecutionException {
+  public List<RawResultIterator> processTableBlocks() throws QueryExecutionException, IOException {
 
     List<RawResultIterator> resultList =
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -124,7 +125,7 @@ public class CarbonCompactionExecutor {
    * @return
    */
   private CarbonIterator<BatchResult> executeBlockList(List<TableBlockInfo> blockList)
-      throws QueryExecutionException {
+      throws QueryExecutionException, IOException {
     queryModel.setTableBlockInfos(blockList);
     this.queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
     return queryExecutor.execute(queryModel);
@@ -165,12 +166,8 @@ public class CarbonCompactionExecutor {
    * @return
    */
   private QueryModel prepareQueryModel(List<TableBlockInfo> blockList) {
-
     QueryModel model = new QueryModel();
-
     model.setTableBlockInfos(blockList);
-    model.setCountStarQuery(false);
-    model.setDetailQuery(true);
     model.setForcedDetailRawQuery(true);
     model.setFilterExpressionResolverTree(null);
 
@@ -188,18 +185,9 @@ public class CarbonCompactionExecutor {
       msrs.add(queryMeasure);
     }
     model.setQueryMeasures(msrs);
-
     model.setQueryId(System.nanoTime() + "");
-
     model.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier());
-
-    model.setAggTable(false);
-    model.setLimit(-1);
-
     model.setTable(carbonTable);
-
-    model.setInMemoryRecordSize(CarbonCommonConstants.COMPACTION_INMEMORY_RECORD_SIZE);
-
     return model;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
index 404dd1a..21bd605 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
@@ -29,14 +29,12 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
 import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.carbon.datastore.block.TaskBlockInfo;
-import org.apache.carbondata.core.carbon.datastore.exception.IndexBuilderException;
 import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.carbon.path.CarbonTablePath;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.CarbonUtilException;
 
 /**
  * Utility Class for the Compaction Flow.
@@ -107,7 +105,7 @@ public class CarbonCompactionUtil {
    * @return
    */
   public static Map<String, List<DataFileFooter>> createDataFileFooterMappingForSegments(
-      List<TableBlockInfo> tableBlockInfoList) throws IndexBuilderException {
+      List<TableBlockInfo> tableBlockInfoList) throws IOException {
 
     Map<String, List<DataFileFooter>> segmentBlockInfoMapping = new HashMap<>();
     for (TableBlockInfo blockInfo : tableBlockInfoList) {
@@ -116,18 +114,12 @@ public class CarbonCompactionUtil {
       DataFileFooter dataFileMatadata = null;
       // check if segId is already present in map
       List<DataFileFooter> metadataList = segmentBlockInfoMapping.get(segId);
-      try {
-        dataFileMatadata = CarbonUtil
-            .readMetadatFile(blockInfo);
-      } catch (CarbonUtilException e) {
-        throw new IndexBuilderException(e);
-      }
+      dataFileMatadata = CarbonUtil.readMetadatFile(blockInfo);
       if (null == metadataList) {
         // if it is not present
         eachSegmentBlocks.add(dataFileMatadata);
         segmentBlockInfoMapping.put(segId, eachSegmentBlocks);
       } else {
-
         // if its already present then update the list.
         metadataList.add(dataFileMatadata);
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index 94fa601..01edf4a 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -17,6 +17,7 @@
 
 package org.apache.carbondata.spark.rdd
 
+import java.io.IOException
 import java.util
 import java.util.{Collections, List}
 
@@ -37,7 +38,7 @@ import org.apache.carbondata.core.carbon.{AbsoluteTableIdentifier, CarbonTableId
 import org.apache.carbondata.core.carbon.datastore.block.{Distributable, SegmentProperties, TaskBlockInfo}
 import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, CarbonUtilException}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit, CarbonMultiBlockSplit}
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
 import org.apache.carbondata.processing.model.CarbonLoadModel
@@ -248,7 +249,7 @@ class CarbonMergerRDD[K, V](
         dataFileFooter = CarbonUtil.readMetadatFile(
             CarbonInputSplit.getTableBlockInfo(carbonInputSplit))
       } catch {
-        case e: CarbonUtilException =>
+        case e: IOException =>
           logError("Exception in preparing the data file footer for compaction " + e.getMessage)
           throw e
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index ec064ed..208f993 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -163,7 +163,6 @@ class TableNewProcessor(cm: TableModel) {
       val colProps = cm.colProps.get.get(colName)
       colProps.asScala.foreach { x => colPropMap.put(x.key, x.value) }
     }
-    columnSchema.setColumnProperties(colPropMap)
     columnSchema.setEncodingList(encoders)
     val colUniqueIdGenerator = CarbonCommonFactory.getColumnUniqueIdGenerator
     val columnUniqueId = colUniqueIdGenerator.generateUniqueId(cm.databaseName,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/integration/spark/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java b/integration/spark/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
index 46e5244..27e6192 100644
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
+++ b/integration/spark/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
@@ -18,6 +18,7 @@
  */
 package org.apache.carbondata.spark.readsupport;
 
+import java.io.IOException;
 import java.sql.Date;
 import java.sql.Timestamp;
 
@@ -35,7 +36,7 @@ import org.apache.spark.unsafe.types.UTF8String;
 public class SparkRowReadSupportImpl extends AbstractDictionaryDecodedReadSupport<Row> {
 
   @Override public void initialize(CarbonColumn[] carbonColumns,
-      AbsoluteTableIdentifier absoluteTableIdentifier) {
+      AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException {
     super.initialize(carbonColumns, absoluteTableIdentifier);
     //can initialize and generate schema here.
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonScan.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonScan.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonScan.scala
index c351c30..405d9d1 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonScan.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonScan.scala
@@ -17,9 +17,6 @@
 
 package org.apache.spark.sql
 
-import java.util
-import java.util.ArrayList
-
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
@@ -29,8 +26,6 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.execution.LeafNode
 import org.apache.spark.sql.hive.CarbonMetastore
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.hadoop.CarbonProjection
 import org.apache.carbondata.scan.model._
 import org.apache.carbondata.spark.CarbonFilters
@@ -51,11 +46,6 @@ case class CarbonScan(
 
   val buildCarbonPlan: CarbonQueryPlan = {
     val plan: CarbonQueryPlan = new CarbonQueryPlan(relationRaw.databaseName, relationRaw.tableName)
-
-    plan.setSortedDimemsions(new util.ArrayList[QueryDimension])
-
-    plan.setOutLocationPath(
-      CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS))
     plan.setQueryId(ocRaw.getConf("queryId", System.nanoTime() + ""))
     processFilterExpressions(plan)
     plan

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala b/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
index dd51184..2c45524 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
@@ -17,15 +17,11 @@
 
 package org.apache.spark.sql
 
-import java.util
-
 import scala.collection.JavaConverters._
 
 import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression}
-import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
+import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression, GenericMutableRow}
 
-import org.apache.carbondata.core.carbon.metadata.encoder.Encoding
 import org.apache.carbondata.scan.expression.{ColumnExpression, ExpressionResult, UnknownExpression}
 import org.apache.carbondata.scan.expression.conditional.ConditionalExpression
 import org.apache.carbondata.scan.expression.exception.FilterUnsupportedException
@@ -121,7 +117,7 @@ class SparkUnknownExpression(var sparkExp: SparkExpression)
 
 
   def getAllColumnListFromExpressionTree(sparkCurrentExp: SparkExpression,
-      list: util.List[ColumnExpression]): util.List[ColumnExpression] = {
+      list: java.util.List[ColumnExpression]): java.util.List[ColumnExpression] = {
     sparkCurrentExp match {
       case carbonBoundRef: CarbonBoundReference => list.add(carbonBoundRef.colExp)
       case _ => sparkCurrentExp.children.foreach(getColumnListFromExpressionTree(_, list))
@@ -129,13 +125,4 @@ class SparkUnknownExpression(var sparkExp: SparkExpression)
     list
   }
 
-  def isDirectDictionaryColumns: Boolean = {
-    val lst = new util.ArrayList[ColumnExpression]()
-    getAllColumnListFromExpressionTree(sparkExp, lst)
-    if (lst.get(0).getCarbonColumn.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-      true
-    } else {
-      false
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/integration/spark2/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
index 3357d6c..a87a26c 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
@@ -18,6 +18,8 @@
  */
 package org.apache.carbondata.spark.readsupport;
 
+import java.io.IOException;
+
 import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
@@ -29,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
 public class SparkRowReadSupportImpl extends AbstractDictionaryDecodedReadSupport<InternalRow> {
 
   @Override public void initialize(CarbonColumn[] carbonColumns,
-      AbsoluteTableIdentifier absoluteTableIdentifier) {
+      AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException {
     super.initialize(carbonColumns, absoluteTableIdentifier);
     //can initialize and generate schema here.
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
index 19c3c9c..895718d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
@@ -42,9 +42,6 @@ case class CarbonScan(
 
   val buildCarbonPlan: CarbonQueryPlan = {
     val plan: CarbonQueryPlan = new CarbonQueryPlan(relationRaw.databaseName, relationRaw.tableName)
-    plan.setSortedDimemsions(new java.util.ArrayList[QueryDimension])
-    plan.setOutLocationPath(
-      CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS))
     processFilterExpressions(plan)
     plan
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
index 3d35837..f3fdc8e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
@@ -43,7 +43,6 @@ import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
 import org.pentaho.di.core.KettleEnvironment;
 import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.exception.KettleXMLException;
 import org.pentaho.di.core.logging.LogLevel;
 import org.pentaho.di.core.logging.LoggingObjectInterface;
 import org.pentaho.di.core.logging.LoggingRegistry;
@@ -83,44 +82,25 @@ public class DataGraphExecuter {
    * @return true if same, false otherwise.
    */
   private boolean checkCSVAndRequestedTableColumns(String[] columnNames, String csvFilePath,
-      String delimiter) {
+      String delimiter) throws IOException {
     return GraphExecutionUtil.checkCSVAndRequestedTableColumns(csvFilePath, columnNames, delimiter);
   }
 
   /**
    * This method returns the Columns names from the schema.
    *
-   * @param schemaInfo
    * @param tableName
    * @return column names array.
    */
-  private String[] getColumnNames(SchemaInfo schemaInfo, String tableName, String partitionId,
-      CarbonDataLoadSchema schema) {
-
+  private String[] getColumnNames(String tableName, CarbonDataLoadSchema schema) {
     Set<String> columnNames = CarbonDataProcessorUtil.getSchemaColumnNames(schema, tableName);
-
-    return columnNames.toArray(new String[columnNames.size()]);
-  }
-
-  /**
-   * This method returns the Columns names for the dimension only from the schema.
-   *
-   * @param schema
-   * @param schemaInfo
-   * @return column names array.
-   */
-  private String[] getDimColumnNames(SchemaInfo schemaInfo, String factTableName,
-      String dimTableName, String partitionId, CarbonDataLoadSchema schema) {
-
-    Set<String> columnNames = GraphExecutionUtil.getDimensionColumnNames(dimTableName, schema);
     return columnNames.toArray(new String[columnNames.size()]);
   }
 
-  private void validateCSV(SchemaInfo schemaInfo, String tableName, CarbonFile f,
-      String partitionId, CarbonDataLoadSchema schema, String delimiter)
-      throws DataLoadingException {
+  private void validateCSV(String tableName, CarbonFile f, CarbonDataLoadSchema schema,
+      String delimiter) throws DataLoadingException, IOException {
 
-    String[] columnNames = getColumnNames(schemaInfo, tableName, partitionId, schema);
+    String[] columnNames = getColumnNames(tableName, schema);
 
     if (!checkCSVAndRequestedTableColumns(columnNames, f.getAbsolutePath(), delimiter)) {
       LOGGER.error(
@@ -134,14 +114,14 @@ public class DataGraphExecuter {
     }
   }
 
-  public void executeGraph(String graphFilePath, List<String> measureColumns, SchemaInfo schemaInfo,
-      String partitionId, CarbonDataLoadSchema schema) throws DataLoadingException {
+  public void executeGraph(String graphFilePath, SchemaInfo schemaInfo, CarbonDataLoadSchema schema)
+      throws DataLoadingException {
 
     //This Method will validate the both fact and dimension csv files.
     if (!schemaInfo.isAutoAggregateRequest() && model.getRddIteratorKey() == null ) {
-      validateCSVFiles(schemaInfo, partitionId, schema);
+      validateCSVFiles(schema);
     }
-    execute(graphFilePath, measureColumns, schemaInfo);
+    execute(graphFilePath, schemaInfo);
   }
 
   /**
@@ -150,7 +130,7 @@ public class DataGraphExecuter {
    * @throws DataLoadingException
    */
 
-  private void execute(String graphFilePath, List<String> measureColumns, SchemaInfo schemaInfo)
+  private void execute(String graphFilePath, SchemaInfo schemaInfo)
       throws DataLoadingException {
 
     //This Method will validate the both fact and dimension csv files.
@@ -178,28 +158,18 @@ public class DataGraphExecuter {
         List<StepMeta> stepsMeta = trans.getTransMeta().getSteps();
         StringBuilder builder = new StringBuilder();
         StringBuilder measuresInCSVFile = new StringBuilder();
-        processCsvInputMeta(measureColumns, stepsMeta, builder, measuresInCSVFile);
+        processCsvInputMeta(stepsMeta, builder, measuresInCSVFile);
         processGetFileNamesMeta(stepsMeta);
 
-        processHadoopFileInputMeta(measureColumns, stepsMeta, builder, measuresInCSVFile);
+        processHadoopFileInputMeta(stepsMeta, builder, measuresInCSVFile);
       }
       setGraphLogLevel();
       trans.execute(null);
       LOGGER.info("Graph execution is started " + graphFilePath);
       trans.waitUntilFinished();
       LOGGER.info("Graph execution is finished.");
-    } catch (KettleXMLException e) {
-      LOGGER.error(e,
-          "Unable to start execution of graph " + e.getMessage());
-      throw new DataLoadingException("Unable to start execution of graph ", e);
-
-    } catch (KettleException e) {
-      LOGGER.error(e,
-          "Unable to start execution of graph " + e.getMessage());
-      throw new DataLoadingException("Unable to start execution of graph ", e);
-    } catch (Throwable e) {
-      LOGGER.error(e,
-          "Unable to start execution of graph " + e.getMessage());
+    } catch (KettleException | IOException e) {
+      LOGGER.error(e, "Unable to start execution of graph " + e.getMessage());
       throw new DataLoadingException("Unable to start execution of graph ", e);
     }
 
@@ -233,14 +203,13 @@ public class DataGraphExecuter {
   }
 
   /**
-   * @param measureColumns
    * @param stepsMeta
    * @param builder
    * @param measuresInCSVFile
    * @throws DataLoadingException
    */
-  private void processHadoopFileInputMeta(List<String> measureColumns, List<StepMeta> stepsMeta,
-      StringBuilder builder, StringBuilder measuresInCSVFile) throws DataLoadingException {
+  private void processHadoopFileInputMeta(List<StepMeta> stepsMeta, StringBuilder builder,
+      StringBuilder measuresInCSVFile) throws DataLoadingException {
     for (StepMeta step : stepsMeta) {
       if (step.getStepMetaInterface() instanceof HadoopFileInputMeta) {
 
@@ -356,14 +325,13 @@ public class DataGraphExecuter {
   }
 
   /**
-   * @param measureColumns
    * @param stepsMeta
    * @param builder
    * @param measuresInCSVFile
    * @throws DataLoadingException
    */
-  private void processCsvInputMeta(List<String> measureColumns, List<StepMeta> stepsMeta,
-      StringBuilder builder, StringBuilder measuresInCSVFile) throws DataLoadingException {
+  private void processCsvInputMeta(List<StepMeta> stepsMeta, StringBuilder builder,
+      StringBuilder measuresInCSVFile) throws DataLoadingException {
     for (StepMeta step : stepsMeta) {
       if (step.getStepMetaInterface() instanceof CsvInputMeta) {
         if (null != model.getCsvFilePath() && model.getRddIteratorKey() == null) {
@@ -419,11 +387,10 @@ public class DataGraphExecuter {
   /**
    * This method will validate the both fact as well as dimension csv files.
    *
-   * @param schemaInfo
+   * @param schema
    * @throws DataLoadingException
    */
-  private void validateCSVFiles(SchemaInfo schemaInfo, String partitionId,
-      CarbonDataLoadSchema schema) throws DataLoadingException {
+  private void validateCSVFiles(CarbonDataLoadSchema schema) throws DataLoadingException {
     // Validate the Fact CSV Files.
     String csvFilePath = model.getCsvFilePath();
     if (csvFilePath != null) {
@@ -445,7 +412,7 @@ public class DataGraphExecuter {
           });
 
           for (CarbonFile f : listFiles) {
-            validateCSV(schemaInfo, model.getTableName(), f, partitionId, schema, ",");
+            validateCSV(model.getTableName(), f, schema, ",");
           }
         } else {
 
@@ -458,12 +425,12 @@ public class DataGraphExecuter {
           }
 
           if (exists) {
-            validateCSV(schemaInfo, model.getTableName(),
-                FileFactory.getCarbonFile(csvFilePath, fileType), partitionId, schema, ",");
+            validateCSV(model.getTableName(),
+                FileFactory.getCarbonFile(csvFilePath, fileType), schema, ",");
           } else {
-            validateCSV(schemaInfo, model.getTableName(), FileFactory
+            validateCSV(model.getTableName(), FileFactory
                 .getCarbonFile(csvFilePath + CarbonCommonConstants.FILE_INPROGRESS_STATUS,
-                    fileType), partitionId, schema, ",");
+                    fileType), schema, ",");
           }
 
         }
@@ -487,8 +454,8 @@ public class DataGraphExecuter {
           try {
             FileFactory.FileType fileType = FileFactory.getFileType(file);
             if (FileFactory.isFileExist(file, fileType)) {
-              validateCSV(schemaInfo, model.getTableName(),
-                  FileFactory.getCarbonFile(file, fileType), partitionId, schema,
+              validateCSV(model.getTableName(),
+                  FileFactory.getCarbonFile(file, fileType), schema,
                   model.getCsvDelimiter());
             }
           } catch (IOException e) {
@@ -500,45 +467,4 @@ public class DataGraphExecuter {
     }
   }
 
-  /**
-   * Validate the dimension csv files.
-   *
-   * @param schema
-   * @param schemaInfo
-   * @param dimFile
-   * @throws DataLoadingException
-   */
-  private void validateDimensionCSV(SchemaInfo schemaInfo, String factTableName,
-      String dimTableName, CarbonFile dimFile, String partitionId, CarbonDataLoadSchema schema,
-      String delimiter) throws DataLoadingException {
-    String[] columnNames =
-        getDimColumnNames(schemaInfo, factTableName, dimTableName, partitionId, schema);
-
-    if (null == columnNames || columnNames.length < 1) {
-      return;
-    }
-    if (!checkAllColumnsPresent(columnNames, dimFile.getAbsolutePath(), delimiter)) {
-      LOGGER.error(
-          "CSV File provided is not proper. Column names in schema and csv header are not same. "
-              + "CSVFile Name : "
-              + dimFile.getName());
-      throw new DataLoadingException(DataProcessorConstants.CSV_VALIDATION_ERRROR_CODE,
-          "Dimension CSV file provided is not proper. Column names in Schema and csv header are "
-              + "not same. CSVFile Name : "
-              + dimFile.getName());
-    }
-
-  }
-
-  /**
-   * Check All the columns are present in the CSV File
-   *
-   * @param dimFilePath
-   * @return
-   */
-  private boolean checkAllColumnsPresent(String[] columnNames, String dimFilePath,
-      String delimiter) {
-    return GraphExecutionUtil.checkCSVAndRequestedTableColumns(dimFilePath, columnNames, delimiter);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
index 2148901..64f21b0 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
@@ -196,7 +196,7 @@ public final class GraphExecutionUtil {
    * @return
    */
   public static boolean checkCSVAndRequestedTableColumns(String csvFilePath, String[] columnNames,
-      String delimiter) {
+      String delimiter) throws IOException {
 
     String readLine = CarbonUtil.readHeader(csvFilePath);
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
index 6d9be67..efe7830 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
@@ -42,7 +42,6 @@ import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.CarbonUtilException;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.processing.newflow.dictionary.DictionaryServerClientDictionary;
 import org.apache.carbondata.processing.newflow.dictionary.DirectDictionary;
@@ -141,11 +140,7 @@ public class PrimitiveDataType implements GenericDataType<Object> {
         Dictionary dictionary = null;
         if (useOnePass) {
           if (CarbonUtil.isFileExistsForGivenColumn(storePath, identifier)) {
-            try {
-              dictionary = cache.get(identifier);
-            } catch (CarbonUtilException e) {
-              throw new RuntimeException(e);
-            }
+            dictionary = cache.get(identifier);
           }
           String threadNo = "initial";
           DictionaryKey dictionaryKey = new DictionaryKey();
@@ -166,7 +161,7 @@ public class PrimitiveDataType implements GenericDataType<Object> {
           dictionaryGenerator = new PreCreatedDictionary(dictionary);
         }
       }
-    } catch (CarbonUtilException e) {
+    } catch (IOException e) {
       throw new RuntimeException(e);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java
index a420d27..824a12e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java
@@ -20,6 +20,7 @@
 package org.apache.carbondata.processing.mdkeygen;
 
 import java.io.File;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -257,8 +258,6 @@ public class MDKeyGenStep extends BaseStep {
   /**
    * This method will be used to get and update the step properties which will
    * required to run this step
-   *
-   * @throws CarbonUtilException
    */
   private boolean setStepConfiguration() {
     this.tableName = meta.getTableName();
@@ -287,7 +286,7 @@ public class MDKeyGenStep extends BaseStep {
     int[] dimLensWithComplex = null;
     try {
       dimLensWithComplex = CarbonUtil.getCardinalityFromLevelMetadataFile(levelCardinalityFilePath);
-    } catch (CarbonUtilException e) {
+    } catch (IOException e) {
       LOGGER.error("Level cardinality file :: " + e.getMessage());
       return false;
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/processing/src/main/java/org/apache/carbondata/processing/newflow/AbstractDataLoadProcessorStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/AbstractDataLoadProcessorStep.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/AbstractDataLoadProcessorStep.java
index cc4da62..0445d84 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/AbstractDataLoadProcessorStep.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/AbstractDataLoadProcessorStep.java
@@ -19,6 +19,7 @@
 
 package org.apache.carbondata.processing.newflow;
 
+import java.io.IOException;
 import java.util.Iterator;
 
 import org.apache.carbondata.common.CarbonIterator;
@@ -56,9 +57,9 @@ public abstract class AbstractDataLoadProcessorStep {
   /**
    * Initialization process for this step.
    *
-   * @throws CarbonDataLoadingException
+   * @throws IOException
    */
-  public abstract void initialize() throws CarbonDataLoadingException;
+  public abstract void initialize() throws IOException;
 
   /**
    * Tranform the data as per the implementation.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/RowConverter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/RowConverter.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/RowConverter.java
index 3b199ab..1180777 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/RowConverter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/RowConverter.java
@@ -18,6 +18,8 @@
  */
 package org.apache.carbondata.processing.newflow.converter;
 
+import java.io.IOException;
+
 import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
 import org.apache.carbondata.processing.newflow.row.CarbonRow;
 
@@ -26,7 +28,7 @@ import org.apache.carbondata.processing.newflow.row.CarbonRow;
  */
 public interface RowConverter {
 
-  void initialize();
+  void initialize() throws IOException;
 
   CarbonRow convert(CarbonRow row) throws CarbonDataLoadingException;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java
index 82a7bc2..6e6243b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java
@@ -19,14 +19,14 @@
 
 package org.apache.carbondata.processing.newflow.converter.impl;
 
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.dictionary.*;
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
@@ -35,7 +35,6 @@ import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.dictionary.client.DictionaryClient;
 import org.apache.carbondata.core.dictionary.generator.key.DictionaryKey;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.CarbonUtilException;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.processing.newflow.DataField;
 import org.apache.carbondata.processing.newflow.converter.BadRecordLogHolder;
@@ -46,9 +45,6 @@ import org.apache.carbondata.processing.newflow.row.CarbonRow;
 
 public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConverterImpl {
 
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DictionaryFieldConverterImpl.class.getName());
-
   private BiDictionary<Integer, Object> dictionaryGenerator;
 
   private int index;
@@ -60,7 +56,8 @@ public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConvert
   public DictionaryFieldConverterImpl(DataField dataField,
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
       CarbonTableIdentifier carbonTableIdentifier, String nullFormat, int index,
-      DictionaryClient client, Boolean useOnePass, String storePath) {
+      DictionaryClient client, Boolean useOnePass, String storePath)
+      throws IOException {
     this.index = index;
     this.carbonDimension = (CarbonDimension) dataField.getColumn();
     this.nullFormat = nullFormat;
@@ -72,12 +69,7 @@ public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConvert
     // if use one pass, use DictionaryServerClientDictionary
     if (useOnePass) {
       if (CarbonUtil.isFileExistsForGivenColumn(storePath, identifier)) {
-        try{
-          dictionary = cache.get(identifier);
-        } catch (CarbonUtilException e) {
-          LOGGER.error(e);
-          throw new RuntimeException(e);
-        }
+        dictionary = cache.get(identifier);
       }
       String threadNo = "initial";
       DictionaryKey dictionaryKey = new DictionaryKey();
@@ -94,13 +86,8 @@ public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConvert
       dictionaryGenerator = new DictionaryServerClientDictionary(dictionary, client,
               dictionaryKey, localCache);
     } else {
-      try {
-        dictionary = cache.get(identifier);
-        dictionaryGenerator = new PreCreatedDictionary(dictionary);
-      } catch (CarbonUtilException e) {
-        LOGGER.error(e);
-        throw new RuntimeException(e);
-      }
+      dictionary = cache.get(identifier);
+      dictionaryGenerator = new PreCreatedDictionary(dictionary);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java
index b065b55..dac8593 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java
@@ -18,6 +18,7 @@
  */
 package org.apache.carbondata.processing.newflow.converter.impl;
 
+import java.io.IOException;
 import java.util.List;
 
 import org.apache.carbondata.core.cache.Cache;
@@ -62,7 +63,8 @@ public class FieldEncoderFactory {
   public FieldConverter createFieldEncoder(DataField dataField,
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
       CarbonTableIdentifier carbonTableIdentifier, int index, String nullFormat,
-      DictionaryClient client, Boolean useOnePass, String storePath) {
+      DictionaryClient client, Boolean useOnePass, String storePath)
+      throws IOException {
     // Converters are only needed for dimensions and measures it return null.
     if (dataField.getColumn().isDimesion()) {
       if (dataField.getColumn().hasEncoding(Encoding.DIRECT_DICTIONARY) &&

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java
index e46b86a..7f7deb1 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java
@@ -18,9 +18,14 @@
  */
 package org.apache.carbondata.processing.newflow.converter.impl;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.*;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.CacheProvider;
@@ -67,7 +72,7 @@ public class RowConverterImpl implements RowConverter {
   }
 
   @Override
-  public void initialize() {
+  public void initialize() throws IOException {
     CacheProvider cacheProvider = CacheProvider.getInstance();
     Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache =
         cacheProvider.createCache(CacheType.REVERSE_DICTIONARY,
@@ -102,10 +107,8 @@ public class RowConverterImpl implements RowConverter {
 
       try {
         dictClient = result.get();
-      } catch (InterruptedException e) {
-        e.printStackTrace();
-      } catch (ExecutionException e) {
-        e.printStackTrace();
+      } catch (InterruptedException | ExecutionException e) {
+        throw new RuntimeException(e);
       }
     }
     for (int i = 0; i < fields.length; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaadc88a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java
index 4504f0c..73e43fd 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java
@@ -163,7 +163,7 @@ public class UnsafeParallelReadMergeSorterImpl implements Sorter {
       CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
           .recordDictionaryValuesTotalTime(parameters.getPartitionID(), System.currentTimeMillis());
       return false;
-    } catch (CarbonSortKeyAndGroupByException e) {
+    } catch (InterruptedException e) {
       throw new CarbonDataLoadingException(e);
     }
   }



Mime
View raw message