http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java 
b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
index 70a515a..31d41b3 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -23,20 +23,13 @@ package org.carbondata.core.util;
 import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.OutputStream;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
@@ -55,9 +48,8 @@ import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
 import 
org.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreDataHolder;
-import org.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreInfo;
 import org.carbondata.core.datastorage.store.columnar.UnBlockIndexer;
 import org.carbondata.core.datastorage.store.compression.MeasureMetaDataModel;
 import org.carbondata.core.datastorage.store.compression.ValueCompressionModel;
@@ -65,12 +57,7 @@ import 
org.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFileFilter;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.keygenerator.mdkey.NumberCompressor;
-import org.carbondata.core.metadata.BlockletInfo;
-import org.carbondata.core.metadata.BlockletInfoColumnar;
-import org.carbondata.core.metadata.SliceMetaData;
 import org.carbondata.core.metadata.ValueEncoderMeta;
-import org.carbondata.core.reader.CarbonFooterReader;
-import org.carbondata.core.vo.ColumnGroupModel;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -131,35 +118,6 @@ public final class CarbonUtil {
     }
   }
 
-  public static File[] getSortedFileList(File[] fileArray) {
-    Arrays.sort(fileArray, new Comparator<File>() {
-      public int compare(File o1, File o2) {
-        try {
-          return o1.getName().compareTo(o2.getName());
-        } catch (Exception e) {
-
-          LOGGER.error(e, "Error while getSortedFile");
-          return 0;
-        }
-      }
-    });
-    return fileArray;
-  }
-
-  public static CarbonFile[] getSortedFileList(CarbonFile[] fileArray) {
-    Arrays.sort(fileArray, new Comparator<CarbonFile>() {
-      public int compare(CarbonFile o1, CarbonFile o2) {
-        try {
-          return o1.getName().compareTo(o2.getName());
-        } catch (Exception e) {
-
-          return o1.getName().compareTo(o2.getName());
-        }
-      }
-    });
-    return fileArray;
-  }
-
   /**
    * @param baseStorePath
    * @return
@@ -425,35 +383,6 @@ public final class CarbonUtil {
    * @param path file path array
    * @throws Exception exception
    */
-  public static void deleteFoldersAndFiles(final String... path) throws 
CarbonUtilException {
-    if (path == null) {
-      return;
-    }
-    try {
-      UserGroupInformation.getLoginUser().doAs(new 
PrivilegedExceptionAction<Void>() {
-
-        @Override public Void run() throws Exception {
-          for (int i = 0; i < path.length; i++) {
-            if (null != path[i]) {
-              deleteRecursive(new File(path[i]));
-            }
-          }
-          return null;
-        }
-      });
-    } catch (IOException e) {
-      throw new CarbonUtilException("Error while deleting the folders and 
files");
-    } catch (InterruptedException e) {
-      throw new CarbonUtilException("Error while deleting the folders and 
files");
-    }
-  }
-
-  /**
-   * This method will be used to delete the folder and files
-   *
-   * @param path file path array
-   * @throws Exception exception
-   */
   public static void deleteFoldersAndFiles(final File... path) throws 
CarbonUtilException {
     try {
       UserGroupInformation.getLoginUser().doAs(new 
PrivilegedExceptionAction<Void>() {
@@ -571,10 +500,10 @@ public final class CarbonUtil {
           boolean isRenameSuccessfull = file.renameTo(newFilePath);
           if (!isRenameSuccessfull) {
             LOGGER.error("Problem renaming the cube :: " + fullPath);
-            c = new DeleteCube(file);
+            c = new DeleteFolderAndFiles(file);
             executorService.submit(c);
           } else {
-            c = new DeleteCube(FileFactory.getCarbonFile(newFilePath, 
fileType));
+            c = new 
DeleteFolderAndFiles(FileFactory.getCarbonFile(newFilePath, fileType));
             executorService.submit(c);
           }
         }
@@ -617,176 +546,6 @@ public final class CarbonUtil {
     }
   }
 
-  /**
-   * This method will be used to read leaf meta data format of meta data will 
be
-   * <entrycount><keylength><keyoffset><measure1length><measure1offset>
-   *
-   * @param file
-   * @param measureCount
-   * @param mdKeySize
-   * @return will return blocklet info which will have all the meta data
-   * related to data file
-   */
-  public static List<BlockletInfo> getBlockletInfo(File file, int 
measureCount, int mdKeySize) {
-    List<BlockletInfo> listOfBlockletInfo =
-        new ArrayList<BlockletInfo>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    String filesLocation = file.getAbsolutePath();
-    long fileSize = file.length();
-    return getBlockletDetails(listOfBlockletInfo, filesLocation, measureCount, 
mdKeySize, fileSize);
-  }
-
-  /**
-   * This method will be used to read leaf meta data format of meta data will 
be
-   * <entrycount><keylength><keyoffset><measure1length><measure1offset>
-   *
-   * @param file
-   * @param measureCount
-   * @param mdKeySize
-   * @return will return blocklet info which will have all the meta data
-   * related to data file
-   */
-  public static List<BlockletInfo> getBlockletInfo(CarbonFile file, int 
measureCount,
-      int mdKeySize) {
-    List<BlockletInfo> listOfNodeInfo =
-        new ArrayList<BlockletInfo>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    String filesLocation = file.getAbsolutePath();
-    long fileSize = file.getSize();
-    return getBlockletDetails(listOfNodeInfo, filesLocation, measureCount, 
mdKeySize, fileSize);
-  }
-
-  /**
-   * @param listOfNodeInfo
-   * @param filesLocation
-   * @param measureCount
-   * @param mdKeySize
-   * @param fileSize
-   * @return
-   */
-  private static List<BlockletInfo> getBlockletDetails(List<BlockletInfo> 
listOfNodeInfo,
-      String filesLocation, int measureCount, int mdKeySize, long fileSize) {
-    long offset = fileSize - CarbonCommonConstants.LONG_SIZE_IN_BYTE;
-    FileHolder fileHolder = 
FileFactory.getFileHolder(FileFactory.getFileType(filesLocation));
-    offset = fileHolder.readDouble(filesLocation, offset);
-    int totalMetaDataLength = (int) (fileSize - 
CarbonCommonConstants.LONG_SIZE_IN_BYTE - offset);
-    ByteBuffer buffer =
-        ByteBuffer.wrap(fileHolder.readByteArray(filesLocation, offset, 
totalMetaDataLength));
-    buffer.rewind();
-    while (buffer.hasRemaining()) {
-      int[] msrLength = new int[measureCount];
-      long[] msrOffset = new long[measureCount];
-      BlockletInfo info = new BlockletInfo();
-      byte[] startKey = new byte[mdKeySize];
-      byte[] endKey = new byte[mdKeySize];
-      info.setFileName(filesLocation);
-      info.setNumberOfKeys(buffer.getInt());
-      info.setKeyLength(buffer.getInt());
-      info.setKeyOffset(buffer.getLong());
-      buffer.get(startKey);
-      buffer.get(endKey);
-      info.setStartKey(startKey);
-      info.setEndKey(endKey);
-      for (int i = 0; i < measureCount; i++) {
-        msrLength[i] = buffer.getInt();
-        msrOffset[i] = buffer.getLong();
-      }
-      info.setMeasureLength(msrLength);
-      info.setMeasureOffset(msrOffset);
-      listOfNodeInfo.add(info);
-    }
-    fileHolder.finish();
-    return listOfNodeInfo;
-  }
-
-  /**
-   * This method will be used to read blocklet meta data format of meta data 
will
-   * be <entrycount><keylength><keyoffset><measure1length><measure1offset>
-   *
-   * @param file
-   * @return will return blocklet info which will have all the meta data
-   * related to leaf file
-   */
-  public static List<BlockletInfoColumnar> getBlockletInfoColumnar(CarbonFile 
file) {
-    List<BlockletInfoColumnar> listOfBlockletInfo =
-        new 
ArrayList<BlockletInfoColumnar>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    String filesLocation = file.getAbsolutePath();
-    long fileSize = file.getSize();
-    return getBlockletInfo(listOfBlockletInfo, filesLocation, fileSize);
-  }
-
-  /**
-   * @param listOfBlockletInfo
-   * @param filesLocation
-   * @param fileSize
-   * @return
-   */
-  private static List<BlockletInfoColumnar> getBlockletInfo(
-      List<BlockletInfoColumnar> listOfBlockletInfo, String filesLocation, 
long fileSize) {
-    long offset = fileSize - CarbonCommonConstants.LONG_SIZE_IN_BYTE;
-    FileHolder fileHolder = 
FileFactory.getFileHolder(FileFactory.getFileType(filesLocation));
-    offset = fileHolder.readDouble(filesLocation, offset);
-    CarbonFooterReader metaDataReader = new CarbonFooterReader(filesLocation, 
offset);
-    try {
-      listOfBlockletInfo = 
CarbonMetadataUtil.convertBlockletInfo(metaDataReader.readFooter());
-    } catch (IOException e) {
-      LOGGER.error("Problem while reading metadata :: " + filesLocation);
-    }
-    for (BlockletInfoColumnar infoColumnar : listOfBlockletInfo) {
-      infoColumnar.setFileName(filesLocation);
-    }
-    return listOfBlockletInfo;
-  }
-
-  /**
-   * This method will be used to read the slice metadata
-   *
-   * @param rsFiles
-   * @return slice meta data
-   * @throws CarbonUtilException
-   */
-  public static SliceMetaData readSliceMetadata(File rsFiles, int 
restructFolderNumber)
-      throws CarbonUtilException {
-    SliceMetaData readObject = null;
-    InputStream stream = null;
-    ObjectInputStream objectInputStream = null;
-    File file = null;
-    try {
-      file = new File(rsFiles + File.separator + 
getSliceMetaDataFileName(restructFolderNumber));
-      stream = new FileInputStream(
-          rsFiles + File.separator + 
getSliceMetaDataFileName(restructFolderNumber));
-      objectInputStream = new ObjectInputStream(stream);
-      readObject = (SliceMetaData) objectInputStream.readObject();
-    } catch (ClassNotFoundException e) {
-      throw new CarbonUtilException(
-          "Problem while reading the slicemeta data file " + 
file.getAbsolutePath(), e);
-    }
-    //
-    catch (IOException e) {
-      throw new CarbonUtilException("Problem while reading the slicemeta data 
file ", e);
-    } finally {
-      closeStreams(objectInputStream, stream);
-    }
-    return readObject;
-  }
-
-  public static void writeSliceMetaDataFile(String path, SliceMetaData 
sliceMetaData,
-      int nextRestructFolder) {
-    OutputStream stream = null;
-    ObjectOutputStream objectOutputStream = null;
-    try {
-      LOGGER.info("Slice Metadata file Path: " + path + '/' + CarbonUtil
-          .getSliceMetaDataFileName(nextRestructFolder));
-      stream = FileFactory
-          .getDataOutputStream(path + File.separator + 
getSliceMetaDataFileName(nextRestructFolder),
-              FileFactory.getFileType(path));
-      objectOutputStream = new ObjectOutputStream(stream);
-      objectOutputStream.writeObject(sliceMetaData);
-    } catch (IOException e) {
-      LOGGER.error(e.getMessage());
-    } finally {
-      closeStreams(objectOutputStream, stream);
-    }
-  }
-
   public static void deleteFiles(File[] intermediateFiles) throws 
CarbonUtilException {
     for (int i = 0; i < intermediateFiles.length; i++) {
       if (!intermediateFiles[i].delete()) {
@@ -795,27 +554,6 @@ public final class CarbonUtil {
     }
   }
 
-  public static ColumnarKeyStoreInfo 
getColumnarKeyStoreInfo(BlockletInfoColumnar blockletInfo,
-      int[] eachBlockSize, ColumnGroupModel colGrpModel) {
-    ColumnarKeyStoreInfo columnarKeyStoreInfo = new ColumnarKeyStoreInfo();
-    columnarKeyStoreInfo.setFilePath(blockletInfo.getFileName());
-    columnarKeyStoreInfo.setIsSorted(blockletInfo.getIsSortedKeyColumn());
-    
columnarKeyStoreInfo.setKeyBlockIndexLength(blockletInfo.getKeyBlockIndexLength());
-    
columnarKeyStoreInfo.setKeyBlockIndexOffsets(blockletInfo.getKeyBlockIndexOffSets());
-    columnarKeyStoreInfo.setKeyBlockLengths(blockletInfo.getKeyLengths());
-    columnarKeyStoreInfo.setKeyBlockOffsets(blockletInfo.getKeyOffSets());
-    columnarKeyStoreInfo.setNumberOfKeys(blockletInfo.getNumberOfKeys());
-    columnarKeyStoreInfo.setSizeOfEachBlock(eachBlockSize);
-    columnarKeyStoreInfo.setNumberCompressor(new 
NumberCompressor(Integer.parseInt(
-        
CarbonProperties.getInstance().getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
-            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL))));
-    columnarKeyStoreInfo.setAggKeyBlock(blockletInfo.getAggKeyBlock());
-    
columnarKeyStoreInfo.setDataIndexMapLength(blockletInfo.getDataIndexMapLength());
-    
columnarKeyStoreInfo.setDataIndexMapOffsets(blockletInfo.getDataIndexMapOffsets());
-    columnarKeyStoreInfo.setHybridStoreModel(colGrpModel);
-    return columnarKeyStoreInfo;
-  }
-
   public static byte[] getKeyArray(ColumnarKeyStoreDataHolder[] 
columnarKeyStoreDataHolder,
       int totalKeySize, int eachKeySize) {
     byte[] completeKeyArray = new byte[totalKeySize];
@@ -918,103 +656,6 @@ public final class CarbonUtil {
     return integers;
   }
 
-  public static String[] getSlices(String storeLocation,
-      FileFactory.FileType fileType) {
-    try {
-      if (!FileFactory.isFileExist(storeLocation, fileType)) {
-        return new String[0];
-      }
-    } catch (IOException e) {
-      LOGGER.error("Error occurred :: " + e.getMessage());
-    }
-    CarbonFile file = FileFactory.getCarbonFile(storeLocation, fileType);
-    CarbonFile[] listFiles = listFiles(file);
-    if (null == listFiles || listFiles.length < 0) {
-      return new String[0];
-    }
-    Arrays.sort(listFiles, new CarbonFileFolderComparator());
-    String[] slices = new String[listFiles.length];
-    for (int i = 0; i < listFiles.length; i++) {
-      slices[i] = listFiles[i].getAbsolutePath();
-    }
-    return slices;
-  }
-
-  /**
-   * @param file
-   * @return
-   */
-  public static CarbonFile[] listFiles(CarbonFile file) {
-    CarbonFile[] listFiles = file.listFiles(new CarbonFileFilter() {
-      @Override public boolean accept(CarbonFile pathname) {
-        return 
pathname.getName().startsWith(CarbonCommonConstants.LOAD_FOLDER) && !pathname
-            .getName().endsWith(CarbonCommonConstants.FILE_INPROGRESS_STATUS);
-      }
-    });
-    return listFiles;
-  }
-
-  public static List<CarbonSliceAndFiles> getSliceAndFilesList(String 
tableName,
-      CarbonFile[] listFiles, FileFactory.FileType fileType) {
-
-    List<CarbonSliceAndFiles> sliceFactFilesList =
-        new ArrayList<CarbonSliceAndFiles>(listFiles.length);
-    if (listFiles.length == 0) {
-      return sliceFactFilesList;
-    }
-
-    CarbonSliceAndFiles sliceAndFiles = null;
-    CarbonFile[] sortedPathForFiles = null;
-    for (int i = 0; i < listFiles.length; i++) {
-      sliceAndFiles = new CarbonSliceAndFiles();
-      sliceAndFiles.setPath(listFiles[i].getAbsolutePath());
-      sortedPathForFiles = getAllFactFiles(sliceAndFiles.getPath(), tableName, 
fileType);
-      if (null != sortedPathForFiles && sortedPathForFiles.length > 0) {
-        Arrays.sort(sortedPathForFiles,
-            new CarbonFileComparator("\\" + 
CarbonCommonConstants.FACT_FILE_EXT));
-        sliceAndFiles.setSliceFactFilesList(sortedPathForFiles);
-        sliceFactFilesList.add(sliceAndFiles);
-      }
-    }
-    return sliceFactFilesList;
-  }
-
-  /**
-   * Below method will be used to get the fact file present in slice
-   *
-   * @param sliceLocation slice location
-   * @return fact files array
-   */
-  public static CarbonFile[] getAllFactFiles(String sliceLocation, final 
String tableName,
-      FileFactory.FileType fileType) {
-    CarbonFile file = FileFactory.getCarbonFile(sliceLocation, fileType);
-    CarbonFile[] files = null;
-    CarbonFile[] updatedFactFiles = null;
-    if (file.isDirectory()) {
-      updatedFactFiles = file.listFiles(new CarbonFileFilter() {
-
-        @Override public boolean accept(CarbonFile pathname) {
-          return ((!pathname.isDirectory()) && 
(pathname.getName().startsWith(tableName))
-              && 
pathname.getName().endsWith(CarbonCommonConstants.FACT_UPDATE_EXTENSION));
-        }
-      });
-
-      if (updatedFactFiles.length != 0) {
-        return updatedFactFiles;
-
-      }
-
-      files = file.listFiles(new CarbonFileFilter() {
-        public boolean accept(CarbonFile pathname) {
-          return ((!pathname.isDirectory()) && 
(pathname.getName().startsWith(tableName))
-              && 
pathname.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT));
-
-        }
-      });
-    }
-    return files;
-  }
-
   /**
    * Read level metadata file and return cardinality
    *
@@ -1049,61 +690,6 @@ public final class CarbonUtil {
     return cardinality;
   }
 
-  public static String getNewAggregateTableName(List<String> tableList, String 
factTableName) {
-    int count = 1;
-    List<String> newTableList = new ArrayList<String>(10);
-    newTableList.addAll(tableList);
-    if (newTableList.contains(factTableName)) {
-      newTableList.remove(factTableName);
-    }
-    if (!newTableList.isEmpty()) {
-      Collections.sort(newTableList, new AggTableComparator());
-      String highestCountAggTableName = newTableList.get(0);
-      count = Integer.parseInt(
-          
highestCountAggTableName.substring(highestCountAggTableName.lastIndexOf("_") + 
1))
-          + count;
-    }
-    return CarbonCommonConstants.AGGREGATE_TABLE_START_TAG + 
CarbonCommonConstants.UNDERSCORE
-        + factTableName + CarbonCommonConstants.UNDERSCORE + count;
-  }
-
-  public static String getRSPath(String schemaName, String cubeName, String 
tableName,
-      String hdfsLocation, int currentRestructNumber) {
-    if (null == hdfsLocation) {
-      hdfsLocation =
-          
CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS);
-    }
-
-    String hdfsStoreLocation = hdfsLocation;
-    hdfsStoreLocation = hdfsStoreLocation + File.separator + schemaName + 
File.separator + cubeName;
-
-    int rsCounter = 
currentRestructNumber/*CarbonUtil.checkAndReturnNextRestructFolderNumber(
-                hdfsStoreLocation, "RS_")*/;
-    if (rsCounter == -1) {
-      rsCounter = 0;
-    }
-    String hdfsLoadedTable =
-        hdfsStoreLocation + File.separator + 
CarbonCommonConstants.RESTRUCTRE_FOLDER + rsCounter
-            + "/" + tableName;
-    return hdfsLoadedTable;
-  }
-
-  public static boolean createRSMetaFile(String metaDataPath, String 
newRSFileName) {
-    String fullFileName = metaDataPath + File.separator + newRSFileName;
-    FileFactory.FileType fileType =
-        FileFactory.getFileType(metaDataPath + File.separator + newRSFileName);
-    try {
-      return FileFactory.createNewFile(fullFileName, fileType);
-    } catch (IOException e) {
-      LOGGER.error("Error while writing RS meta file : " + fullFileName + 
e.getMessage());
-      return false;
-    }
-  }
-
-  public static String getSliceMetaDataFileName(int restructFolderNumber) {
-    return CarbonCommonConstants.SLICE_METADATA_FILENAME + "." + 
restructFolderNumber;
-  }
-
   public static void writeLevelCardinalityFile(String loadFolderLoc, String 
tableName,
       int[] dimCardinality) throws KettleException {
     String levelCardinalityFilePath = loadFolderLoc + File.separator +
@@ -1140,32 +726,6 @@ public final class CarbonUtil {
     }
   }
 
-  public static SliceMetaData readSliceMetaDataFile(String path) {
-    SliceMetaData readObject = null;
-    InputStream stream = null;
-    ObjectInputStream objectInputStream = null;
-    //
-    try {
-      stream = FileFactory.getDataInputStream(path, 
FileFactory.getFileType(path));
-      objectInputStream = new ObjectInputStream(stream);
-      readObject = (SliceMetaData) objectInputStream.readObject();
-    } catch (ClassNotFoundException e) {
-      LOGGER.error(e);
-    } catch (FileNotFoundException e) {
-      LOGGER.error("@@@@@ SliceMetaData File is missing @@@@@ :" + path);
-    } catch (IOException e) {
-      LOGGER.error("@@@@@ Error while reading SliceMetaData File @@@@@ :" + 
path);
-    } finally {
-      closeStreams(objectInputStream, stream);
-    }
-    return readObject;
-  }
-
-  public static SliceMetaData readSliceMetaDataFile(String folderPath, int 
currentRestructNumber) {
-    String path = folderPath + '/' + 
getSliceMetaDataFileName(currentRestructNumber);
-    return readSliceMetaDataFile(path);
-  }
-
   /**
    * From beeline if a delimeter is passed as \001, in code we get it as
    * escaped string as \\001. So this method will unescape the slash again and
@@ -1248,26 +808,6 @@ public final class CarbonUtil {
   }
 
   /**
-   * This method will read the retry time interval for loading level files in
-   * memory
-   *
-   * @return
-   */
-  public static long getRetryIntervalForLoadingLevelFile() {
-    long retryInterval = 0;
-    try {
-      retryInterval = Long.parseLong(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.CARBON_LOAD_LEVEL_RETRY_INTERVAL,
-              CarbonCommonConstants.CARBON_LOAD_LEVEL_RETRY_INTERVAL_DEFAULT));
-    } catch (NumberFormatException e) {
-      retryInterval = Long.parseLong(CarbonProperties.getInstance()
-          
.getProperty(CarbonCommonConstants.CARBON_LOAD_LEVEL_RETRY_INTERVAL_DEFAULT));
-    }
-    retryInterval = retryInterval * 1000;
-    return retryInterval;
-  }
-
-  /**
    * Below method will be used to get the aggregator type
    * CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE will return when value is 
double measure
    * CarbonCommonConstants.BYTE_VALUE_MEASURE will be returned when value is 
byte array
@@ -1500,12 +1040,11 @@ public final class CarbonUtil {
   /**
    * Thread to delete the cubes
    *
-   * @author m00258959
    */
-  private static final class DeleteCube implements Callable<Void> {
+  private static final class DeleteFolderAndFiles implements Callable<Void> {
     private CarbonFile file;
 
-    private DeleteCube(CarbonFile file) {
+    private DeleteFolderAndFiles(CarbonFile file) {
       this.file = file;
     }
 
@@ -1516,34 +1055,6 @@ public final class CarbonUtil {
 
   }
 
-  private static class CarbonFileComparator implements Comparator<CarbonFile> {
-    /**
-     * File extension
-     */
-    private String fileExt;
-
-    public CarbonFileComparator(String fileExt) {
-      this.fileExt = fileExt;
-    }
-
-    @Override public int compare(CarbonFile file1, CarbonFile file2) {
-      String firstFileName = file1.getName().split(fileExt)[0];
-      String secondFileName = file2.getName().split(fileExt)[0];
-      int lastIndexOfO1 = firstFileName.lastIndexOf('_');
-      int lastIndexOfO2 = secondFileName.lastIndexOf('_');
-      int f1 = 0;
-      int f2 = 0;
-
-      try {
-        f1 = Integer.parseInt(firstFileName.substring(lastIndexOfO1 + 1));
-        f2 = Integer.parseInt(secondFileName.substring(lastIndexOfO2 + 1));
-      } catch (NumberFormatException nfe) {
-        return -1;
-      }
-      return (f1 < f2) ? -1 : (f1 == f2 ? 0 : 1);
-    }
-  }
-
   /**
    * class to sort aggregate folder list in descending order
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java 
b/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
deleted file mode 100644
index b3b9d37..0000000
--- a/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * 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.carbondata.core.vo;
-
-public class ColumnGroupModel {
-
-  /**
-   * cardinality all dimension
-   */
-  private int[] colGroupCardinality;
-
-  /**
-   * number of columns in columnar block
-   */
-  private int[] columnSplit;
-
-  /**
-   * total number of columns
-   */
-  private int noOfColumnsStore;
-
-  /**
-   * whether given index is columnar or not
-   * true: columnar
-   * false: row block
-   */
-  private boolean[] columnarStore;
-
-  /**
-   * column groups
-   * e.g
-   * {{0,1,2},3,4,{5,6}}
-   */
-  private int[][] columnGroups;
-
-  /**
-   * @return cardinality of column groups
-   */
-  public int[] getColumnGroupCardinality() {
-    return colGroupCardinality;
-  }
-
-  /**
-   * set columngroup cardinality
-   *
-   * @param columnGroupCardinality
-   */
-  public void setColumnGroupCardinality(int[] columnGroupCardinality) {
-    this.colGroupCardinality = columnGroupCardinality;
-  }
-
-  /**
-   * return columnSplit
-   *
-   * @return
-   */
-  public int[] getColumnSplit() {
-    return columnSplit;
-  }
-
-  /**
-   * set columnSplit
-   *
-   * @param split
-   */
-  public void setColumnSplit(int[] split) {
-    this.columnSplit = split;
-  }
-
-  /**
-   * @return no of columnar block
-   */
-  public int getNoOfColumnStore() {
-    return this.noOfColumnsStore;
-  }
-
-  /**
-   * set no of columnar block
-   *
-   * @param noOfColumnsStore
-   */
-  public void setNoOfColumnStore(int noOfColumnsStore) {
-    this.noOfColumnsStore = noOfColumnsStore;
-  }
-
-  /**
-   * it's an identifier for row block or single column block
-   *
-   * @param columnarStore
-   */
-  public void setColumnarStore(boolean[] columnarStore) {
-    this.columnarStore = columnarStore;
-  }
-
-  /**
-   * set column groups
-   *
-   * @param columnGroups
-   */
-  public void setColumnGroup(int[][] columnGroups) {
-    this.columnGroups = columnGroups;
-  }
-
-  /**
-   * check if given column group is columnar
-   *
-   * @param colGroup
-   * @return true if given block is columnar
-   */
-  public boolean isColumnar(int colGroup) {
-    return columnarStore[colGroup];
-  }
-
-  /**
-   * @return columngroups
-   */
-  public int[][] getColumnGroup() {
-    return this.columnGroups;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/writer/CarbonDataWriter.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/core/writer/CarbonDataWriter.java 
b/core/src/main/java/org/carbondata/core/writer/CarbonDataWriter.java
deleted file mode 100644
index 0f31c45..0000000
--- a/core/src/main/java/org/carbondata/core/writer/CarbonDataWriter.java
+++ /dev/null
@@ -1,482 +0,0 @@
-/*
- * 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.carbondata.core.writer;
-
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
-import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.file.manager.composite.FileData;
-import org.carbondata.core.file.manager.composite.IFileManagerComposite;
-import org.carbondata.core.metadata.BlockletInfo;
-import org.carbondata.core.util.CarbonProperties;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.core.writer.exception.CarbonDataWriterException;
-
-public class CarbonDataWriter {
-  /**
-   * Attribute for Carbon LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CarbonDataWriter.class.getName());
-  /**
-   * table name
-   */
-  private String tableName;
-  /**
-   * data file size;
-   */
-  private long fileSizeInBytes;
-  /**
-   * measure count
-   */
-  private int measureCount;
-  /**
-   * this will be used for holding blocklet metadata
-   */
-  private List<BlockletInfo> blockletInfoList;
-  /**
-   * current size of file
-   */
-  private long currentFileSize;
-  /**
-   * leaf metadata size
-   */
-  private int leafMetaDataSize;
-  /**
-   * file count will be used to give sequence number to the data file
-   */
-  private int fileCount;
-  /**
-   * filename format
-   */
-  private String fileNameFormat;
-  /**
-   * file name
-   */
-  private String fileName;
-  /**
-   * File manager
-   */
-  private IFileManagerComposite fileManager;
-  /**
-   * Store Location
-   */
-  private String storeLocation;
-  /**
-   * fileExtension
-   */
-  private String fileExtension;
-  /**
-   * isNewFileCreationRequired
-   */
-  private boolean isNewFileCreationRequired;
-  /**
-   * isInProgressExtrequired
-   */
-  private boolean isInProgressExtrequired;
-  /**
-   * fileDataOutStream
-   */
-  private DataOutputStream fileDataOutStream;
-  /**
-   * metadataOffset for maintaining the offset of pagination file.
-   */
-  private int metadataOffset;
-
-  /**
-   * CarbonDataWriter constructor to initialize all the instance variables
-   * required for wrting the data i to the file
-   *
-   * @param storeLocation current store location
-   * @param measureCount  total number of measures
-   * @param mdKeyLength   mdkey length
-   * @param tableName     table name
-   */
-  public CarbonDataWriter(String storeLocation, int measureCount, int 
mdKeyLength, String tableName,
-      String fileExtension, boolean isNewFileCreationRequired, boolean 
isInProgressExtrequired) {
-    // measure count
-    this.measureCount = measureCount;
-    // table name
-    this.tableName = tableName;
-
-    this.storeLocation = storeLocation;
-    this.fileExtension = fileExtension;
-    // create the carbon file format
-    this.fileNameFormat =
-        storeLocation + File.separator + this.tableName + '_' + "{0}" + 
this.fileExtension;
-
-    this.leafMetaDataSize = CarbonCommonConstants.INT_SIZE_IN_BYTE * (2 + 
measureCount)
-        + CarbonCommonConstants.LONG_SIZE_IN_BYTE * (measureCount + 1) + (2 * 
mdKeyLength);
-    this.blockletInfoList = new 
ArrayList<BlockletInfo>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    // get max file size;
-    this.fileSizeInBytes = Long.parseLong(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.MAX_FILE_SIZE,
-            CarbonCommonConstants.MAX_FILE_SIZE_DEFAULT_VAL))
-        * CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR
-        * CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR * 1L;
-    this.isNewFileCreationRequired = isNewFileCreationRequired;
-    this.isInProgressExtrequired = isInProgressExtrequired;
-  }
-
-  /**
-   * This method will be used to initialize the channel
-   *
-   * @throws CarbonDataWriterException
-   */
-  public void initChannel() throws CarbonDataWriterException {
-    // update the filename with new new sequence
-    // increment the file sequence counter
-    initFileCount();
-    if (this.isInProgressExtrequired) {
-      this.fileName = MessageFormat.format(this.fileNameFormat, this.fileCount)
-          + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
-      FileData fileData = new FileData(this.fileName, this.storeLocation);
-      fileManager.add(fileData);
-    } else {
-      this.fileName = MessageFormat.format(this.fileNameFormat, 
this.fileCount);
-    }
-    this.fileCount++;
-    try {
-      // open stream for new data file
-      this.fileDataOutStream = FileFactory
-          .getDataOutputStream(this.fileName, 
FileFactory.getFileType(this.fileName), (short) 1);
-    } catch (FileNotFoundException fileNotFoundException) {
-      throw new CarbonDataWriterException("Problem while getting the writer 
for Leaf File",
-          fileNotFoundException);
-    } catch (IOException e) {
-      throw new CarbonDataWriterException("Problem while getting the writer 
for Leaf File", e);
-    }
-  }
-
-  /**
-   * Method will be used to close the open stream
-   */
-  public CarbonFile closeChannle() {
-    CarbonUtil.closeStreams(this.fileDataOutStream);
-
-    CarbonFile carbonFile = FileFactory.getCarbonFile(fileName, 
FileFactory.getFileType(fileName));
-
-    if (!carbonFile.renameTo(fileName.substring(0, 
this.fileName.lastIndexOf('.')))) {
-      LOGGER.info("file renaming failed from _0.querymerged to _0");
-    }
-
-    return carbonFile;
-  }
-
-  private int initFileCount() {
-    int fileCnt = 0;
-    File[] dataFiles = new File(storeLocation).listFiles(new FileFilter() {
-
-      @Override public boolean accept(File file) {
-        if (!file.isDirectory() && file.getName().startsWith(tableName) && 
file.getName()
-            .contains(fileExtension)) {
-          return true;
-        }
-        return false;
-      }
-    });
-    if (dataFiles != null && dataFiles.length > 0) {
-      Arrays.sort(dataFiles);
-      String fileName = dataFiles[dataFiles.length - 1].getName();
-      try {
-        fileCnt =
-            Integer.parseInt(fileName.substring(fileName.lastIndexOf('_') + 
1).split("\\.")[0]);
-      } catch (NumberFormatException ex) {
-        fileCnt = 0;
-      }
-      fileCnt++;
-    }
-    return fileCnt;
-  }
-
-  /**
-   * This method will be used to update the file channel with new file; new
-   * file will be created once existing file reached the file size limit This
-   * method will first check whether existing file size is exceeded the file
-   * size limit if yes then write the blocklet metadata to file then set the
-   * current file size to 0 close the existing file channel get the new file
-   * name and get the channel for new file
-   *
-   * @throws CarbonDataWriterException if any problem
-   */
-  private void updateBlockletFileChannel() throws CarbonDataWriterException {
-    // get the current file size exceeding the file size threshold
-    if (currentFileSize >= fileSizeInBytes) {
-      // write meta data to end of the existing file
-      writeBlockletMetaDataToFile();
-      // set the current file size;
-      this.currentFileSize = 0;
-      // close the current open file channel
-      CarbonUtil.closeStreams(fileDataOutStream);
-      // initialize the new channel
-      initChannel();
-    }
-  }
-
-  /**
-   * This method will be used to write leaf data to file
-   * file format
-   * <key><measure1><measure2>....
-   *
-   * @param keyArray   key array
-   * @param dataArray  measure array
-   * @param entryCount number of entries
-   * @param startKey   start key of leaf
-   * @param endKey     end key of leaf
-   * @throws CarbonDataWriterException
-   * @throws CarbonDataWriterException throws new CarbonDataWriterException if 
any problem
-   */
-  public void writeDataToFile(byte[] keyArray, byte[][] dataArray, int 
entryCount, byte[] startKey,
-      byte[] endKey) throws CarbonDataWriterException {
-    if (this.isNewFileCreationRequired) {
-      updateBlockletFileChannel();
-    }
-    // total measure length;
-    int totalMsrArraySize = 0;
-    // current measure length;
-    int currentMsrLenght = 0;
-    int[] msrLength = new int[this.measureCount];
-
-    // calculate the total size required for all the measure and get the
-    // each measure size
-    for (int i = 0; i < dataArray.length; i++) {
-      currentMsrLenght = dataArray[i].length;
-      totalMsrArraySize += currentMsrLenght;
-      msrLength[i] = currentMsrLenght;
-    }
-    byte[] writableDataArray = new byte[totalMsrArraySize];
-
-    // start position will be used for adding the measure in
-    // writableDataArray after adding measure increment the start position
-    // by added measure length which will be used for next measure start
-    // position
-    int startPosition = 0;
-    for (int i = 0; i < dataArray.length; i++) {
-      System.arraycopy(dataArray[i], 0, writableDataArray, startPosition, 
dataArray[i].length);
-      startPosition += msrLength[i];
-    }
-    writeDataToFile(keyArray, writableDataArray, msrLength, entryCount, 
startKey, endKey);
-  }
-
-  /**
-   * This method will be used to write leaf data to file
-   * file format
-   * <key><measure1><measure2>....
-   *
-   * @param keyArray   key array
-   * @param dataArray  measure array
-   * @param entryCount number of entries
-   * @param startKey   start key of leaf
-   * @param endKey     end key of leaf
-   * @throws CarbonDataWriterException
-   * @throws CarbonDataWriterException throws new CarbonDataWriterException if 
any problem
-   */
-  public void writeDataToFile(byte[] keyArray, byte[] dataArray, int[] 
msrLength, int entryCount,
-      byte[] startKey, byte[] endKey) throws CarbonDataWriterException {
-    int keySize = keyArray.length;
-    // write data to leaf file and get its offset
-    long offset = writeDataToFile(keyArray, dataArray);
-
-    // get the blocklet info for currently added blocklet
-    BlockletInfo blockletInfo =
-        getBlockletInfo(keySize, msrLength, offset, entryCount, startKey, 
endKey);
-    // add blocklet info to list
-    this.blockletInfoList.add(blockletInfo);
-    // calculate the current size of the file
-    this.currentFileSize +=
-        keySize + dataArray.length + (blockletInfoList.size() * 
this.leafMetaDataSize)
-            + CarbonCommonConstants.LONG_SIZE_IN_BYTE;
-  }
-
-  /**
-   * This method will be used to get the blocklet metadata
-   *
-   * @param keySize    key size
-   * @param msrLength  measure length array
-   * @param offset     current offset
-   * @param entryCount total number of rows in leaf
-   * @param startKey   start key of leaf
-   * @param endKey     end key of leaf
-   * @return BlockletInfo - leaf metadata
-   */
-  private BlockletInfo getBlockletInfo(int keySize, int[] msrLength, long 
offset, int entryCount,
-      byte[] startKey, byte[] endKey) {
-    // create the info object for leaf entry
-    BlockletInfo info = new BlockletInfo();
-    // add total entry count
-    info.setNumberOfKeys(entryCount);
-
-    // add the key array length
-    info.setKeyLength(keySize);
-
-    // add key offset
-    info.setKeyOffset(offset);
-
-    // increment the current offset by adding key length to get the measure
-    // offset position
-    // format of metadata will be
-    // 
<entrycount>,<keylenght>,<keyoffset>,<msr1lenght><msr1offset><msr2length><msr2offset>
-    offset += keySize;
-
-    // add measure length
-    info.setMeasureLength(msrLength);
-
-    long[] msrOffset = new long[this.measureCount];
-
-    for (int i = 0; i < this.measureCount; i++) {
-      msrOffset[i] = offset;
-      // now increment the offset by adding measure length to get the next
-      // measure offset;
-      offset += msrLength[i];
-    }
-    // add measure offset
-    info.setMeasureOffset(msrOffset);
-    // set startkey
-    info.setStartKey(startKey);
-    // set end key
-    info.setEndKey(endKey);
-    // return leaf metadata
-    return info;
-  }
-
-  /**
-   * This method is responsible for writing blocklet to the data file
-   *
-   * @param keyArray     mdkey array
-   * @param measureArray measure array
-   * @return file offset offset is the current position of the file
-   * @throws CarbonDataWriterException if will throw CarbonDataWriterException 
when any thing
-   *                                   goes wrong while while writing the leaf 
file
-   */
-  private long writeDataToFile(byte[] keyArray, byte[] measureArray)
-      throws CarbonDataWriterException {
-    long offset = metadataOffset;
-    try {
-      metadataOffset += keyArray.length + measureArray.length;
-      this.fileDataOutStream.write(keyArray);
-      this.fileDataOutStream.write(measureArray);
-    } catch (IOException exception) {
-      throw new CarbonDataWriterException("Problem in writing carbon file: ", 
exception);
-    }
-    // return the offset, this offset will be used while reading the file in
-    // engine side to get from which position to start reading the file
-    return offset;
-  }
-
-  /**
-   * This method will write metadata at the end of file file format
-   * <KeyArray><measure1><measure2> <KeyArray><measure1><measure2>
-   * <KeyArray><measure1><measure2> <KeyArray><measure1><measure2>
-   * <entrycount>
-   * <keylength><keyoffset><measure1length><measure1offset><measure2length
-   * ><measure2offset>
-   *
-   * @throws CarbonDataWriterException throw CarbonDataWriterException when 
problem in
-   *                                   writing the meta data to file
-   */
-  public void writeBlockletMetaDataToFile() throws CarbonDataWriterException {
-    ByteBuffer buffer = null;
-    int[] msrLength = null;
-    long[] msroffset = null;
-    try {
-      // get the current position of the file, this will be used for
-      // reading the file meta data, meta data start position in file will
-      // be this position
-      for (BlockletInfo info : this.blockletInfoList) {
-        // get the measure length array
-        msrLength = info.getMeasureLength();
-        // get the measure offset array
-        msroffset = info.getMeasureOffset();
-        // allocate total size for buffer
-        buffer = ByteBuffer.allocate(this.leafMetaDataSize);
-        // add entry count
-        buffer.putInt(info.getNumberOfKeys());
-        // add key length
-        buffer.putInt(info.getKeyLength());
-        // add key offset
-        buffer.putLong(info.getKeyOffset());
-        // set the start key
-        buffer.put(info.getStartKey());
-        // set the end key
-        buffer.put(info.getEndKey());
-        // add each measure length and its offset
-        for (int i = 0; i < this.measureCount; i++) {
-          buffer.putInt(msrLength[i]);
-          buffer.putLong(msroffset[i]);
-        }
-        // flip the buffer
-        buffer.flip();
-        // write metadat to file
-        this.fileDataOutStream.write(buffer.array());
-      }
-      // create new for adding the offset of meta data
-      // write offset to file
-      this.fileDataOutStream.writeLong(metadataOffset);
-    } catch (IOException exception) {
-      throw new CarbonDataWriterException("Problem while writing the carbon 
file: ", exception);
-    }
-    // create new blocklet info list for new file
-    this.blockletInfoList = new 
ArrayList<BlockletInfo>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-  }
-
-  /**
-   * This method will be used to get the leaf meta list size
-   *
-   * @return list size
-   */
-  public int getMetaListSize() {
-    return blockletInfoList.size();
-  }
-
-  public void setFileManager(IFileManagerComposite fileManager) {
-    this.fileManager = fileManager;
-  }
-
-  /**
-   * getFileCount
-   *
-   * @return int
-   */
-  public int getFileCount() {
-    return fileCount;
-  }
-
-  /**
-   * setFileCount
-   *
-   * @param fileCount void
-   */
-  public void setFileCount(int fileCount) {
-    this.fileCount = fileCount;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java 
b/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java
index 6e04714..eea9988 100644
--- a/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java
+++ b/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java
@@ -18,7 +18,7 @@
  */
 package org.carbondata.scan.executor;
 
-import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.scan.executor.exception.QueryExecutionException;
 import org.carbondata.scan.model.QueryModel;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java 
b/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
index 8232567..edb7675 100644
--- 
a/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
+++ 
b/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
@@ -20,7 +20,7 @@ package org.carbondata.scan.executor.impl;
 
 import java.util.List;
 
-import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.scan.executor.exception.QueryExecutionException;
 import org.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.carbondata.scan.model.QueryModel;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
 
b/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
index 1ce0a36..8659e82 100644
--- 
a/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
+++ 
b/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
@@ -2,7 +2,7 @@ package org.carbondata.scan.executor.impl;
 
 import java.util.List;
 
-import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.scan.executor.exception.QueryExecutionException;
 import org.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.carbondata.scan.model.QueryModel;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
 
b/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
index e91e6de..def2725 100644
--- 
a/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
+++ 
b/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
@@ -90,10 +90,6 @@ public class RestructureFilterResolverImpl implements 
FilterResolverIntf {
           } else {
             dimColumnResolvedFilterInfo
                 
.setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-            // dimColumnResolvedFilterInfo
-            // 
.setNeedCompressedData(info.getSlices().get(info.getCurrentSliceIndex())
-            // 
.getDataCache(info.getFactTableName()).getAggKeyBlock()[columnExpression.getDim()
-            // .getOrdinal()]);
             dimColumnResolvedFilterInfo.setFilterValues(
                 FilterUtil.getFilterListForRS(right, columnExpression, 
defaultValue, surrogate));
           }
@@ -115,10 +111,6 @@ public class RestructureFilterResolverImpl implements 
FilterResolverIntf {
           } else {
             dimColumnResolvedFilterInfo
                 
.setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-            // dimColumnResolvedFilterInfo
-            // 
.setNeedCompressedData(info.getSlices().get(info.getCurrentSliceIndex())
-            // 
.getDataCache(info.getFactTableName()).getAggKeyBlock()[columnExpression.getDim()
-            // .getOrdinal()]);
             dimColumnResolvedFilterInfo.setFilterValues(
                 FilterUtil.getFilterListForRS(left, columnExpression, 
defaultValue, surrogate));
           }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
 
b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
index e3c7cd5..e385bec 100644
--- 
a/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
+++ 
b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
@@ -18,11 +18,11 @@
  */
 package org.carbondata.scan.processor;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.datastore.DataRefNode;
 import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.scan.collector.ScannedResultCollector;
 import org.carbondata.scan.collector.impl.ListBasedResultCollector;
 import org.carbondata.scan.executor.exception.QueryExecutionException;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java 
b/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
index 0b36ab5..7cecf64 100644
--- a/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
+++ b/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
@@ -19,8 +19,8 @@
 
 package org.carbondata.scan.processor;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.core.carbon.datastore.DataRefNode;
-import org.carbondata.core.iterator.CarbonIterator;
 
 /**
  * Below class will be used to iterate over data block

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/result/BatchResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/BatchResult.java 
b/core/src/main/java/org/carbondata/scan/result/BatchResult.java
index dc14060..c5e1182 100644
--- a/core/src/main/java/org/carbondata/scan/result/BatchResult.java
+++ b/core/src/main/java/org/carbondata/scan/result/BatchResult.java
@@ -21,7 +21,7 @@ package org.carbondata.scan.result;
 
 import java.util.NoSuchElementException;
 
-import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.common.CarbonIterator;
 
 /**
  * Below class holds the query result

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
 
b/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
index 2356a9f..1b49bcc 100644
--- 
a/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
+++ 
b/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
@@ -20,6 +20,7 @@ package org.carbondata.scan.result.iterator;
 
 import java.util.List;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.datastore.DataRefNode;
@@ -28,7 +29,6 @@ import 
org.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.FileHolder;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.carbondata.scan.model.QueryModel;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java 
b/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java
index 5cc4f1e..63a7a96 100644
--- 
a/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java
+++ 
b/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java
@@ -19,7 +19,7 @@
 
 package org.carbondata.scan.result.iterator;
 
-import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.scan.result.BatchResult;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java 
b/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
index 8c028b2..62849ee 100644
--- 
a/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
+++ 
b/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
@@ -18,10 +18,10 @@
  */
 package org.carbondata.scan.result.iterator;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.scan.result.BatchRawResult;
 import org.carbondata.scan.wrappers.ByteArrayWrapper;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/test/java/org/carbondata/core/locks/LocalFileLockTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/carbondata/core/locks/LocalFileLockTest.java 
b/core/src/test/java/org/carbondata/core/locks/LocalFileLockTest.java
deleted file mode 100644
index cfdc50e..0000000
--- a/core/src/test/java/org/carbondata/core/locks/LocalFileLockTest.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- *
- */
-package org.carbondata.core.locks;
-
-import java.io.File;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.pentaho.di.core.util.Assert;
-
-/**
- * Test class to test the functionality of the local file locking.
- *
- * @author Administrator
- */
-public class LocalFileLockTest {
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Before public void setUp() throws Exception {
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @After public void tearDown() throws Exception {
-  }
-
-  @Test public void testingLocalFileLockingByAcquiring2Locks() {
-
-    LocalFileLock localLock1 =
-        new LocalFileLock((new File(".").getAbsolutePath()) + 
"/src/test/resources",
-            LockUsage.METADATA_LOCK);
-    Assert.assertTrue(localLock1.lock());
-    LocalFileLock localLock2 =
-        new LocalFileLock((new File(".").getAbsolutePath()) + 
"/src/test/resources",
-            LockUsage.METADATA_LOCK);
-    Assert.assertTrue(!localLock2.lock());
-
-    Assert.assertTrue(localLock1.unlock());
-    Assert.assertTrue(localLock2.lock());
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java 
b/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java
deleted file mode 100644
index 3bf3993..0000000
--- a/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/**
- *
- */
-package org.carbondata.core.locks;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.ServerSocket;
-import java.util.Properties;
-
-import org.carbondata.core.util.CarbonProperties;
-
-import mockit.NonStrictExpectations;
-import org.apache.zookeeper.server.ServerConfig;
-import org.apache.zookeeper.server.ZooKeeperServerMain;
-import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * @author Administrator
- */
-public class ZooKeeperLockingTest {
-
-  int freePort;
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Before public void setUp() throws Exception {
-    Properties startupProperties = new Properties();
-    startupProperties.setProperty("dataDir", (new 
File("./target").getAbsolutePath()));
-    startupProperties.setProperty("dataLogDir", (new 
File("./target").getAbsolutePath()));
-    freePort = findFreePort();
-    startupProperties.setProperty("clientPort", "" + freePort);
-    QuorumPeerConfig quorumConfiguration = new QuorumPeerConfig();
-    try {
-      quorumConfiguration.parseProperties(startupProperties);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-
-    final ZooKeeperServerMain zooKeeperServer = new ZooKeeperServerMain();
-    final ServerConfig configuration = new ServerConfig();
-    configuration.readFrom(quorumConfiguration);
-    new Thread() {
-      public void run() {
-        try {
-          zooKeeperServer.runFromConfig(configuration);
-        } catch (IOException e) {
-          System.out.println("ZooKeeper failure");
-        }
-      }
-    }.start();
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @After public void tearDown() throws Exception {
-  }
-
-  @Test public void testZooKeeperLockingByTryingToAcquire2Locks()
-      throws IllegalArgumentException, IllegalAccessException, 
NoSuchFieldException,
-      SecurityException {
-
-    final CarbonProperties cp = CarbonProperties.getInstance();
-    new NonStrictExpectations(cp) {
-      {
-        cp.getProperty("/CarbonLocks");
-        result = "/carbontests";
-        cp.getProperty("spark.deploy.zookeeper.url");
-        result = "127.0.0.1:" + freePort;
-      }
-    };
-
-    ZookeeperInit zki = ZookeeperInit.getInstance("127.0.0.1:" + freePort);
-
-    ZooKeeperLocking zkl =
-        new 
ZooKeeperLocking("D:/carbondata/examples/target/store/default/t3/Metadata",
-            LockUsage.METADATA_LOCK);
-    Assert.assertTrue(zkl.lock());
-
-    ZooKeeperLocking zk2 = new ZooKeeperLocking(
-        "D:/carbondata/examples/target/store/default/t3/Metadata", 
LockUsage.METADATA_LOCK);
-    Assert.assertTrue(!zk2.lock());
-
-    Assert.assertTrue(zkl.unlock());
-    Assert.assertTrue(zk2.lock());
-    Assert.assertTrue(zk2.unlock());
-  }
-
-  /**
-   * For finding the free port available.
-   *
-   * @return
-   */
-  private static int findFreePort() {
-    ServerSocket socket = null;
-    try {
-      socket = new ServerSocket(0);
-      socket.setReuseAddress(true);
-      int port = socket.getLocalPort();
-      try {
-        socket.close();
-      } catch (IOException e) {
-        // Ignore IOException on close()
-      }
-      return port;
-    } catch (Exception e) {
-      // Ignore
-    } finally {
-      if (socket != null) {
-        try {
-          socket.close();
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    }
-    return 2181;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java 
b/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
index 774ad14..f0f996c 100644
--- a/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
+++ b/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
@@ -5,9 +5,9 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.core.cache.dictionary.Dictionary;
 import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.hadoop.readsupport.CarbonReadSupport;
 import org.carbondata.scan.executor.QueryExecutorFactory;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git 
a/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java 
b/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
index 1190a9d..c5a8163 100644
--- a/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
+++ b/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
@@ -58,9 +58,9 @@ import 
org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import 
org.carbondata.core.datastorage.store.fileperations.AtomicFileOperations;
-import 
org.carbondata.core.datastorage.store.fileperations.AtomicFileOperationsImpl;
-import org.carbondata.core.datastorage.store.fileperations.FileWriteOperation;
+import org.carbondata.lcm.fileoperations.AtomicFileOperations;
+import org.carbondata.lcm.fileoperations.AtomicFileOperationsImpl;
+import org.carbondata.lcm.fileoperations.FileWriteOperation;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.load.BlockDetails;
 import org.carbondata.core.load.LoadMetadataDetails;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
----------------------------------------------------------------------
diff --git 
a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
 
b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
index 9ede388..e2cd277 100644
--- 
a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
+++ 
b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.cache.dictionary.Dictionary;
@@ -35,7 +36,6 @@ import 
org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.scan.executor.QueryExecutor;
 import org.carbondata.scan.executor.QueryExecutorFactory;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
----------------------------------------------------------------------
diff --git 
a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
 
b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
index f83ebf6..6e6f4b9 100644
--- 
a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
+++ 
b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
@@ -38,11 +38,11 @@ import 
org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.util.ByteUtil;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.core.vo.ColumnGroupModel;
 import org.carbondata.processing.datatypes.GenericDataType;
 import org.carbondata.processing.merger.exeception.SliceMergerException;
 import org.carbondata.processing.store.CarbonDataFileAttributes;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git 
a/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
 
b/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
index a6615ed..8119d28 100644
--- 
a/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
+++ 
b/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
@@ -40,9 +40,6 @@ import 
org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import 
org.carbondata.core.datastorage.store.fileperations.AtomicFileOperations;
-import 
org.carbondata.core.datastorage.store.fileperations.AtomicFileOperationsImpl;
-import org.carbondata.core.datastorage.store.fileperations.FileWriteOperation;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFileFilter;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
@@ -51,6 +48,9 @@ import org.carbondata.core.load.LoadMetadataDetails;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.CarbonUtilException;
+import org.carbondata.lcm.fileoperations.AtomicFileOperations;
+import org.carbondata.lcm.fileoperations.AtomicFileOperationsImpl;
+import org.carbondata.lcm.fileoperations.FileWriteOperation;
 import org.carbondata.processing.api.dataloader.DataLoadModel;
 import org.carbondata.processing.api.dataloader.SchemaInfo;
 import org.carbondata.processing.csvload.DataGraphExecuter;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
 
b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 3f4dac5..476f25d 100644
--- 
a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ 
b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -47,9 +47,9 @@ import 
org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
 import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.datastorage.store.impl.FileFactory
-import org.carbondata.core.locks.{CarbonLockFactory, LockUsage}
 import org.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.carbondata.integration.spark.merger.CompactionType
+import org.carbondata.lcm.locks.{CarbonLockFactory, LockUsage}
 import org.carbondata.lcm.status.SegmentStatusManager
 import org.carbondata.spark.exception.MalformedCarbonCommandException
 import org.carbondata.spark.load._
@@ -1145,6 +1145,7 @@ private[sql] case class AlterTable(
 
 /**
  * Command for the compaction in alter table command
+ *
  * @param alterTableModel
  */
 private[sql] case class AlterTableCompaction(alterTableModel: AlterTableModel) 
extends

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
 
b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
index 3e929a0..dc44f28 100644
--- 
a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
+++ 
b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
@@ -47,11 +47,11 @@ import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile
 import org.carbondata.core.datastorage.store.impl.FileFactory
 import org.carbondata.core.datastorage.store.impl.FileFactory.FileType
-import org.carbondata.core.locks.ZookeeperInit
 import org.carbondata.core.reader.ThriftReader
 import org.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.carbondata.core.writer.ThriftWriter
 import org.carbondata.format.{SchemaEvolutionEntry, TableInfo}
+import org.carbondata.lcm.locks.ZookeeperInit
 import org.carbondata.spark.util.CarbonScalaUtil.CarbonSparkUtil
 
 case class MetaData(var cubesMeta: ArrayBuffer[TableMeta])
@@ -456,10 +456,6 @@ class CarbonMetastoreCatalog(hive: HiveContext, val 
storePath: String, client: C
     dimArray
   }
 
-  def getAggregateTableName(carbonTable: CarbonTable, factTableName: String): 
String = {
-    CarbonUtil.getNewAggregateTableName(carbonTable.getAggregateTablesName, 
factTableName)
-  }
-
   /**
    * Shows all schemas which has Database name like
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
 
b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 87b0952..f770fc7 100644
--- 
a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ 
b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -23,14 +23,14 @@ import java.util.concurrent.{Executors, ExecutorService, 
Future}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ListBuffer
-import scala.util.control.Breaks._
 import scala.util.Random
+import scala.util.control.Breaks._
 
 import org.apache.hadoop.conf.{Configurable, Configuration}
 import org.apache.hadoop.mapreduce.Job
 import org.apache.hadoop.mapreduce.lib.input.FileSplit
 import org.apache.spark.{Logging, Partition, SparkContext, SparkEnv}
-import org.apache.spark.sql.{CarbonEnv, CarbonRelation, SQLContext}
+import org.apache.spark.sql.{CarbonEnv, SQLContext}
 import org.apache.spark.sql.execution.command.{AlterTableModel, 
CompactionModel, Partitioner}
 import org.apache.spark.util.{FileUtils, SplitUtils}
 
@@ -41,9 +41,9 @@ import org.carbondata.core.carbon.metadata.CarbonMetadata
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.load.{BlockDetails, LoadMetadataDetails}
-import org.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
-import org.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.carbondata.core.util.CarbonUtil
 import org.carbondata.integration.spark.merger.{CompactionCallable, 
CompactionType}
+import org.carbondata.lcm.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
 import org.carbondata.lcm.status.SegmentStatusManager
 import org.carbondata.processing.util.CarbonDataProcessorUtil
 import org.carbondata.spark._

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git 
a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala 
b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
index 84a362b..0de2d1a 100644
--- 
a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
+++ 
b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -27,9 +27,9 @@ import org.apache.hadoop.mapreduce.Job
 import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
 import org.apache.spark.rdd.RDD
 
+import org.carbondata.common.CarbonIterator
 import org.carbondata.common.logging.LogServiceFactory
 import org.carbondata.core.carbon.datastore.block.TableBlockInfo
-import org.carbondata.core.iterator.CarbonIterator
 import org.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
 import org.carbondata.scan.executor.QueryExecutorFactory
 import org.carbondata.scan.expression.Expression

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperations.java
----------------------------------------------------------------------
diff --git 
a/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperations.java
 
b/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperations.java
new file mode 100644
index 0000000..14b39ed
--- /dev/null
+++ 
b/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperations.java
@@ -0,0 +1,33 @@
+/*
+ * 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.carbondata.lcm.fileoperations;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+public interface AtomicFileOperations {
+
+  DataInputStream openForRead() throws IOException;
+
+  void close() throws IOException;
+
+  DataOutputStream openForWrite(FileWriteOperation operation) throws 
IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperationsImpl.java
----------------------------------------------------------------------
diff --git 
a/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperationsImpl.java
 
b/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperationsImpl.java
new file mode 100644
index 0000000..b402eaa
--- /dev/null
+++ 
b/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperationsImpl.java
@@ -0,0 +1,87 @@
+/*
+ * 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.carbondata.lcm.fileoperations;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
+import org.carbondata.core.datastorage.store.impl.FileFactory;
+import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
+
+public class AtomicFileOperationsImpl implements AtomicFileOperations {
+
+  private String filePath;
+
+  private FileType fileType;
+
+  private String tempWriteFilePath;
+
+  private DataOutputStream dataOutStream;
+
+  public AtomicFileOperationsImpl(String filePath, FileType fileType) {
+    this.filePath = filePath;
+
+    this.fileType = fileType;
+  }
+
+  @Override public DataInputStream openForRead() throws IOException {
+    return FileFactory.getDataInputStream(filePath, fileType);
+  }
+
+  @Override public DataOutputStream openForWrite(FileWriteOperation operation) 
throws IOException {
+
+    filePath = filePath.replace("\\", "/");
+
+    tempWriteFilePath = filePath + 
CarbonCommonConstants.TEMPWRITEFILEEXTENSION;
+
+    if (FileFactory.isFileExist(tempWriteFilePath, fileType)) {
+      FileFactory.getCarbonFile(tempWriteFilePath, fileType).delete();
+    }
+
+    FileFactory.createNewFile(tempWriteFilePath, fileType);
+
+    dataOutStream = FileFactory.getDataOutputStream(tempWriteFilePath, 
fileType);
+
+    return dataOutStream;
+
+  }
+
+  /* (non-Javadoc)
+   * @see 
com.huawei.unibi.carbon.datastorage.store.fileperations.AtomicFileOperations#close()
+   */
+  @Override public void close() throws IOException {
+
+    if (null != dataOutStream) {
+      dataOutStream.close();
+
+      CarbonFile tempFile = FileFactory.getCarbonFile(tempWriteFilePath, 
fileType);
+
+      if (!tempFile.renameForce(filePath)) {
+        throw new IOException("temporary file renaming failed, src="
+            + tempFile.getPath() + ", dest=" + filePath);
+      }
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/fileoperations/FileWriteOperation.java
----------------------------------------------------------------------
diff --git 
a/processing/src/main/java/org/carbondata/lcm/fileoperations/FileWriteOperation.java
 
b/processing/src/main/java/org/carbondata/lcm/fileoperations/FileWriteOperation.java
new file mode 100644
index 0000000..d8832af
--- /dev/null
+++ 
b/processing/src/main/java/org/carbondata/lcm/fileoperations/FileWriteOperation.java
@@ -0,0 +1,25 @@
+/*
+ * 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.carbondata.lcm.fileoperations;
+
+public enum FileWriteOperation {
+
+  APPEND, OVERWRITE
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/locks/AbstractCarbonLock.java
----------------------------------------------------------------------
diff --git 
a/processing/src/main/java/org/carbondata/lcm/locks/AbstractCarbonLock.java 
b/processing/src/main/java/org/carbondata/lcm/locks/AbstractCarbonLock.java
new file mode 100644
index 0000000..ee76932
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/locks/AbstractCarbonLock.java
@@ -0,0 +1,77 @@
+/*
+ * 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.carbondata.lcm.locks;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.CarbonProperties;
+
+/**
+ * This is the abstract class of the lock implementations.This handles the
+ * retrying part of the locking.
+ */
+public abstract class AbstractCarbonLock implements ICarbonLock {
+  private int retryCount;
+
+  private int retryTimeout;
+
+  public abstract boolean lock();
+
+  /**
+   * API for enabling the locking of file with retries.
+   */
+  public boolean lockWithRetries() {
+    try {
+      for (int i = 0; i < retryCount; i++) {
+        if (lock()) {
+          return true;
+        } else {
+          Thread.sleep(retryTimeout * 1000L);
+        }
+      }
+    } catch (InterruptedException e) {
+      return false;
+    }
+    return false;
+  }
+
+  /**
+   * Initializes the retry count and retry timeout.
+   * This will determine how many times to retry to acquire lock and the retry 
timeout.
+   */
+  protected void initRetry() {
+    String retries = CarbonProperties.getInstance()
+        
.getProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_LOAD_METADATA_LOCK);
+    try {
+      retryCount = Integer.parseInt(retries);
+    } catch (NumberFormatException e) {
+      retryCount = 
CarbonCommonConstants.NUMBER_OF_TRIES_FOR_LOAD_METADATA_LOCK_DEFAULT;
+    }
+
+    String maxTimeout = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.MAX_TIMEOUT_FOR_LOAD_METADATA_LOCK);
+    try {
+      retryTimeout = Integer.parseInt(maxTimeout);
+    } catch (NumberFormatException e) {
+      retryTimeout = 
CarbonCommonConstants.MAX_TIMEOUT_FOR_LOAD_METADATA_LOCK_DEFAULT;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/locks/CarbonLockFactory.java
----------------------------------------------------------------------
diff --git 
a/processing/src/main/java/org/carbondata/lcm/locks/CarbonLockFactory.java 
b/processing/src/main/java/org/carbondata/lcm/locks/CarbonLockFactory.java
new file mode 100644
index 0000000..8fc1cd5
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/locks/CarbonLockFactory.java
@@ -0,0 +1,72 @@
+/*
+ * 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.carbondata.lcm.locks;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.CarbonProperties;
+
+/**
+ * This class is a Lock factory class which is used to provide lock objects.
+ * Using this lock object client can request the lock and unlock.
+ */
+public class CarbonLockFactory {
+
+  /**
+   * lockTypeConfigured to check if zookeeper feature is enabled or not for 
carbon.
+   */
+  private static String lockTypeConfigured;
+
+  static {
+    CarbonLockFactory.updateZooKeeperLockingStatus();
+  }
+
+  /**
+   * This method will determine the lock type.
+   *
+   * @param location
+   * @param lockUsage
+   * @return
+   */
+  public static ICarbonLock getCarbonLockObj(String location, LockUsage 
lockUsage) {
+    switch (lockTypeConfigured.toUpperCase()) {
+      case CarbonCommonConstants.CARBON_LOCK_TYPE_LOCAL:
+        return new LocalFileLock(location, lockUsage);
+
+      case CarbonCommonConstants.CARBON_LOCK_TYPE_ZOOKEEPER:
+        return new ZooKeeperLocking(location, lockUsage);
+
+      case CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS:
+        return new HdfsFileLock(location, lockUsage);
+
+      default:
+        throw new UnsupportedOperationException("Not supported the lock type");
+    }
+
+  }
+
+  /**
+   * This method will set the zookeeper status whether zookeeper to be used 
for locking or not.
+   */
+  private static void updateZooKeeperLockingStatus() {
+    lockTypeConfigured = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.LOCK_TYPE, 
CarbonCommonConstants.LOCK_TYPE_DEFAULT);
+
+  }
+
+}


Reply via email to