You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2017/11/14 17:50:38 UTC

[16/18] carbondata git commit: [CARBONDATA-1573] [Integration] Support Database Location Configuration while Creating Database/ Support Creation of carbon Table in the database location

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
index 4d13462..de0711b 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
@@ -353,7 +353,7 @@ public class QueryUtil {
             absoluteTableIdentifier.getCarbonTableIdentifier(), tableProvider);
     CacheProvider cacheProvider = CacheProvider.getInstance();
     Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache = cacheProvider
-        .createCache(CacheType.FORWARD_DICTIONARY, absoluteTableIdentifier.getStorePath());
+        .createCache(CacheType.FORWARD_DICTIONARY);
     List<Dictionary> columnDictionaryList =
         forwardDictionaryCache.getAll(dictionaryColumnUniqueIdentifiers);
     Map<String, Dictionary> columnDictionaryMap = new HashMap<>(columnDictionaryList.size());
@@ -384,30 +384,30 @@ public class QueryUtil {
       CarbonDimension dimension = CarbonMetadata.getInstance()
           .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnId);
       if (dimension != null) {
-        CarbonTableIdentifier newCarbonTableIdentifier;
+        AbsoluteTableIdentifier newCarbonTableIdentifier;
         ColumnIdentifier columnIdentifier;
         if (null != dimension.getColumnSchema().getParentColumnTableRelations() && !dimension
             .getColumnSchema().getParentColumnTableRelations().isEmpty()) {
-          newCarbonTableIdentifier = getTableIdentifierForColumn(dimension);
+          newCarbonTableIdentifier = getTableIdentifierForColumn(dimension,
+              carbonTable.getAbsoluteTableIdentifier());
           columnIdentifier = new ColumnIdentifier(
               dimension.getColumnSchema().getParentColumnTableRelations().get(0).getColumnId(),
               dimension.getColumnProperties(), dimension.getDataType());
         } else {
-          newCarbonTableIdentifier = carbonTableIdentifier;
+          newCarbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier();
           columnIdentifier = dimension.getColumnIdentifier();
         }
-        CarbonTablePath newCarbonTablePath = CarbonStorePath
-            .getCarbonTablePath(carbonTable.getStorePath(), newCarbonTableIdentifier);
-
         dictionaryColumnUniqueIdentifiers.add(
             new DictionaryColumnUniqueIdentifier(newCarbonTableIdentifier, columnIdentifier,
-                dimension.getDataType(), newCarbonTablePath));
+                dimension.getDataType(),
+                CarbonStorePath.getCarbonTablePath(newCarbonTableIdentifier)));
       }
     }
     return dictionaryColumnUniqueIdentifiers;
   }
 
-  public static CarbonTableIdentifier getTableIdentifierForColumn(CarbonDimension carbonDimension) {
+  public static AbsoluteTableIdentifier getTableIdentifierForColumn(CarbonDimension carbonDimension,
+      AbsoluteTableIdentifier absoluteTableIdentifier) {
     String parentTableName =
         carbonDimension.getColumnSchema().getParentColumnTableRelations().get(0)
             .getRelationIdentifier().getTableName();
@@ -416,7 +416,11 @@ public class QueryUtil {
             .getRelationIdentifier().getDatabaseName();
     String parentTableId = carbonDimension.getColumnSchema().getParentColumnTableRelations().get(0)
         .getRelationIdentifier().getTableId();
-    return new CarbonTableIdentifier(parentDatabaseName, parentTableName, parentTableId);
+    CarbonTableIdentifier carbonTableIdentifier =
+        new CarbonTableIdentifier(parentDatabaseName, parentTableName, parentTableId);
+    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
+    String newTablePath = CarbonUtil.getNewTablePath(carbonTablePath, carbonTableIdentifier);
+    return new AbsoluteTableIdentifier(newTablePath, carbonTableIdentifier);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
index f67b20c..bb2fe7d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
@@ -1227,26 +1227,26 @@ public final class FilterUtil {
   }
 
   /**
-   * @param tableIdentifier
    * @param carbonDimension
    * @param tableProvider
    * @return
    */
-  public static Dictionary getForwardDictionaryCache(AbsoluteTableIdentifier tableIdentifier,
+  public static Dictionary getForwardDictionaryCache(
+      AbsoluteTableIdentifier absoluteTableIdentifier,
       CarbonDimension carbonDimension, TableProvider tableProvider) throws IOException {
     CarbonTablePath carbonTablePath = null;
     if (null != tableProvider) {
       CarbonTable carbonTable =
-          tableProvider.getCarbonTable(tableIdentifier.getCarbonTableIdentifier());
+          tableProvider.getCarbonTable(absoluteTableIdentifier.getCarbonTableIdentifier());
       carbonTablePath =
           CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
     }
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(tableIdentifier.getCarbonTableIdentifier(),
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
             carbonDimension.getColumnIdentifier(), carbonDimension.getDataType(), carbonTablePath);
     CacheProvider cacheProvider = CacheProvider.getInstance();
     Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache =
-        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, tableIdentifier.getStorePath());
+        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);
     // get the forward dictionary object
     return forwardDictionaryCache.get(dictionaryColumnUniqueIdentifier);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
index 95030d3..8dd61b1 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
@@ -106,7 +106,7 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
     this.recorder = queryModel.getStatisticsRecorder();
     this.blockExecutionInfos = infos;
     this.fileReader = FileFactory.getFileHolder(
-        FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getStorePath()));
+        FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getTablePath()));
     this.fileReader.setQueryId(queryModel.getQueryId());
     this.execService = execService;
     intialiseInfos();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/service/DictionaryService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/service/DictionaryService.java b/core/src/main/java/org/apache/carbondata/core/service/DictionaryService.java
index 672aa83..01b6e79 100644
--- a/core/src/main/java/org/apache/carbondata/core/service/DictionaryService.java
+++ b/core/src/main/java/org/apache/carbondata/core/service/DictionaryService.java
@@ -17,7 +17,6 @@
 package org.apache.carbondata.core.service;
 
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReader;
 import org.apache.carbondata.core.reader.CarbonDictionaryReader;
 import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
@@ -32,59 +31,46 @@ public interface DictionaryService {
   /**
    * get dictionary writer
    *
-   * @param carbonTableIdentifier
    * @param dictionaryColumnUniqueIdentifier
-   * @param carbonStorePath
    * @return
    */
-  CarbonDictionaryWriter getDictionaryWriter(CarbonTableIdentifier carbonTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String carbonStorePath);
+  CarbonDictionaryWriter getDictionaryWriter(
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
 
   /**
    * get dictionary sort index writer
    *
-   * @param carbonTableIdentifier
    * @param dictionaryColumnUniqueIdentifier
-   * @param carbonStorePath
    * @return
    */
   CarbonDictionarySortIndexWriter getDictionarySortIndexWriter(
-      CarbonTableIdentifier carbonTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String carbonStorePath);
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
 
   /**
    * get dictionary metadata reader
    *
-   * @param carbonTableIdentifier
    * @param dictionaryColumnUniqueIdentifier
-   * @param carbonStorePath
    * @return
    */
   CarbonDictionaryMetadataReader getDictionaryMetadataReader(
-      CarbonTableIdentifier carbonTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String carbonStorePath);
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
 
   /**
    * get dictionary reader
    *
-   * @param carbonTableIdentifier
    * @param dictionaryColumnUniqueIdentifier
-   * @param carbonStorePath
    * @return
    */
-  CarbonDictionaryReader getDictionaryReader(CarbonTableIdentifier carbonTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String carbonStorePath);
+  CarbonDictionaryReader getDictionaryReader(
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
 
   /**
    * get dictionary sort index reader
    *
-   * @param carbonTableIdentifier
    * @param dictionaryColumnUniqueIdentifier
-   * @param carbonStorePath
    * @return
    */
   CarbonDictionarySortIndexReader getDictionarySortIndexReader(
-      CarbonTableIdentifier carbonTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String carbonStorePath);
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/service/PathService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/service/PathService.java b/core/src/main/java/org/apache/carbondata/core/service/PathService.java
index 9735217..664d045 100644
--- a/core/src/main/java/org/apache/carbondata/core/service/PathService.java
+++ b/core/src/main/java/org/apache/carbondata/core/service/PathService.java
@@ -17,7 +17,7 @@
 package org.apache.carbondata.core.service;
 
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 /**
@@ -26,11 +26,10 @@ import org.apache.carbondata.core.util.path.CarbonTablePath;
 public interface PathService {
 
   /**
-   * @param storeLocation
-   * @param tableIdentifier
+   * @param absoluteTableIdentifier
    * @param dictionaryColumnUniqueIdentifier
    * @return store path related to tables
    */
-  CarbonTablePath getCarbonTablePath(String storeLocation, CarbonTableIdentifier tableIdentifier,
+  CarbonTablePath getCarbonTablePath(AbsoluteTableIdentifier absoluteTableIdentifier,
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/service/impl/DictionaryFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/service/impl/DictionaryFactory.java b/core/src/main/java/org/apache/carbondata/core/service/impl/DictionaryFactory.java
index 1dc22ba..83ef25f 100644
--- a/core/src/main/java/org/apache/carbondata/core/service/impl/DictionaryFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/service/impl/DictionaryFactory.java
@@ -17,7 +17,6 @@
 package org.apache.carbondata.core.service.impl;
 
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReader;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
 import org.apache.carbondata.core.reader.CarbonDictionaryReader;
@@ -40,76 +39,56 @@ public class DictionaryFactory implements DictionaryService {
   /**
    * get dictionary writer
    *
-   * @param carbonTableIdentifier
    * @param dictionaryColumnUniqueIdentifier
-   * @param carbonStorePath
    * @return
    */
   @Override public CarbonDictionaryWriter getDictionaryWriter(
-      CarbonTableIdentifier carbonTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String carbonStorePath) {
-    return new CarbonDictionaryWriterImpl(carbonStorePath, carbonTableIdentifier,
-        dictionaryColumnUniqueIdentifier);
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
+    return new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
   }
 
   /**
    * get dictionary sort index writer
    *
-   * @param carbonTableIdentifier
    * @param dictionaryColumnUniqueIdentifier
-   * @param carbonStorePath
    * @return
    */
   @Override public CarbonDictionarySortIndexWriter getDictionarySortIndexWriter(
-      CarbonTableIdentifier carbonTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String carbonStorePath) {
-    return new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier,
-        dictionaryColumnUniqueIdentifier, carbonStorePath);
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
+    return new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
   }
 
   /**
    * get dictionary metadata reader
    *
-   * @param carbonTableIdentifier
    * @param dictionaryColumnUniqueIdentifier
-   * @param carbonStorePath
    * @return
    */
   @Override public CarbonDictionaryMetadataReader getDictionaryMetadataReader(
-      CarbonTableIdentifier carbonTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String carbonStorePath) {
-    return new CarbonDictionaryMetadataReaderImpl(carbonStorePath, carbonTableIdentifier,
-        dictionaryColumnUniqueIdentifier);
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
+    return new CarbonDictionaryMetadataReaderImpl(dictionaryColumnUniqueIdentifier);
   }
 
   /**
    * get dictionary reader
    *
-   * @param carbonTableIdentifier
    * @param dictionaryColumnUniqueIdentifier
-   * @param carbonStorePath
    * @return
    */
   @Override public CarbonDictionaryReader getDictionaryReader(
-      CarbonTableIdentifier carbonTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String carbonStorePath) {
-    return new CarbonDictionaryReaderImpl(carbonStorePath, carbonTableIdentifier,
-        dictionaryColumnUniqueIdentifier);
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
+    return new CarbonDictionaryReaderImpl(dictionaryColumnUniqueIdentifier);
   }
 
   /**
    * get dictionary sort index reader
    *
-   * @param carbonTableIdentifier
    * @param dictionaryColumnUniqueIdentifier
-   * @param carbonStorePath
    * @return
    */
   @Override public CarbonDictionarySortIndexReader getDictionarySortIndexReader(
-      CarbonTableIdentifier carbonTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String carbonStorePath) {
-    return new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier,
-        dictionaryColumnUniqueIdentifier, carbonStorePath);
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
+    return new CarbonDictionarySortIndexReaderImpl(dictionaryColumnUniqueIdentifier);
   }
 
   public static DictionaryService getInstance() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java b/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java
index e7e9ebe..cb4ca25 100644
--- a/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java
@@ -17,7 +17,7 @@
 package org.apache.carbondata.core.service.impl;
 
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.service.PathService;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -30,19 +30,18 @@ public class PathFactory implements PathService {
   private static PathService pathService = new PathFactory();
 
   /**
-   * @param storeLocation
-   * @param tableIdentifier
+   * @param absoluteTableIdentifier
    * @param
    * @return store path related to tables
    */
-  @Override public CarbonTablePath getCarbonTablePath(String storeLocation,
-      CarbonTableIdentifier tableIdentifier,
+  @Override public CarbonTablePath getCarbonTablePath(
+      AbsoluteTableIdentifier absoluteTableIdentifier,
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
     if (null != dictionaryColumnUniqueIdentifier && null != dictionaryColumnUniqueIdentifier
         .getCarbonTablePath()) {
       return dictionaryColumnUniqueIdentifier.getCarbonTablePath();
     }
-    return CarbonStorePath.getCarbonTablePath(storeLocation, tableIdentifier);
+    return CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
   }
 
   public static PathService getInstance() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index a42239f..e3dbfed 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -70,8 +70,7 @@ public class SegmentStatusManager {
    * @return
    */
   public ICarbonLock getTableStatusLock() {
-    return CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier.getCarbonTableIdentifier(),
-            LockUsage.TABLE_STATUS_LOCK);
+    return CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier, LockUsage.TABLE_STATUS_LOCK);
   }
 
   /**
@@ -80,7 +79,7 @@ public class SegmentStatusManager {
   public static long getTableStatusLastModifiedTime(AbsoluteTableIdentifier identifier)
       throws IOException {
     String tableStatusPath = CarbonStorePath
-        .getCarbonTablePath(identifier.getStorePath(), identifier.getCarbonTableIdentifier())
+        .getCarbonTablePath(identifier.getTablePath(), identifier.getCarbonTableIdentifier())
         .getTableStatusFilePath();
     if (!FileFactory.isFileExist(tableStatusPath, FileFactory.getFileType(tableStatusPath))) {
       return 0L;
@@ -104,7 +103,7 @@ public class SegmentStatusManager {
     List<String> listOfInvalidSegments = new ArrayList<>(10);
     List<String> listOfStreamSegments = new ArrayList<>(10);
     CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+            .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
                     absoluteTableIdentifier.getCarbonTableIdentifier());
     String dataPath = carbonTablePath.getTableStatusFilePath();
     DataInputStream dataInputStream = null;
@@ -267,9 +266,9 @@ public class SegmentStatusManager {
       List<String> loadIds, String tableFolderPath) throws Exception {
     CarbonTableIdentifier carbonTableIdentifier = identifier.getCarbonTableIdentifier();
     ICarbonLock carbonDeleteSegmentLock =
-        CarbonLockFactory.getCarbonLockObj(carbonTableIdentifier, LockUsage.DELETE_SEGMENT_LOCK);
+        CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.DELETE_SEGMENT_LOCK);
     ICarbonLock carbonTableStatusLock =
-        CarbonLockFactory.getCarbonLockObj(carbonTableIdentifier, LockUsage.TABLE_STATUS_LOCK);
+        CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.TABLE_STATUS_LOCK);
     String tableDetails =
         carbonTableIdentifier.getDatabaseName() + "." + carbonTableIdentifier.getTableName();
     List<String> invalidLoadIds = new ArrayList<String>(0);
@@ -278,7 +277,7 @@ public class SegmentStatusManager {
         LOG.info("Delete segment lock has been successfully acquired");
 
         CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(
-            identifier.getStorePath(), identifier.getCarbonTableIdentifier());
+            identifier.getTablePath(), identifier.getCarbonTableIdentifier());
         String dataLoadLocation = carbonTablePath.getTableStatusFilePath();
         LoadMetadataDetails[] listOfLoadFolderDetailsArray = null;
         if (!FileFactory.isFileExist(dataLoadLocation, FileFactory.getFileType(dataLoadLocation))) {
@@ -349,9 +348,9 @@ public class SegmentStatusManager {
       String loadDate, String tableFolderPath, Long loadStartTime) throws Exception {
     CarbonTableIdentifier carbonTableIdentifier = identifier.getCarbonTableIdentifier();
     ICarbonLock carbonDeleteSegmentLock =
-        CarbonLockFactory.getCarbonLockObj(carbonTableIdentifier, LockUsage.DELETE_SEGMENT_LOCK);
+        CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.DELETE_SEGMENT_LOCK);
     ICarbonLock carbonTableStatusLock =
-        CarbonLockFactory.getCarbonLockObj(carbonTableIdentifier, LockUsage.TABLE_STATUS_LOCK);
+        CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.TABLE_STATUS_LOCK);
     String tableDetails =
         carbonTableIdentifier.getDatabaseName() + "." + carbonTableIdentifier.getTableName();
     List<String> invalidLoadTimestamps = new ArrayList<String>(0);
@@ -360,7 +359,7 @@ public class SegmentStatusManager {
         LOG.info("Delete segment lock has been successfully acquired");
 
         CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(
-            identifier.getStorePath(), identifier.getCarbonTableIdentifier());
+            identifier.getTablePath(), identifier.getCarbonTableIdentifier());
         String dataLoadLocation = carbonTablePath.getTableStatusFilePath();
         LoadMetadataDetails[] listOfLoadFolderDetailsArray = null;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
index 982c598..df7eedd 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
@@ -76,7 +76,7 @@ public class SegmentUpdateStatusManager {
    */
   public SegmentUpdateStatusManager(AbsoluteTableIdentifier absoluteTableIdentifier) {
     this.absoluteTableIdentifier = absoluteTableIdentifier;
-    carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+    carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
         absoluteTableIdentifier.getCarbonTableIdentifier());
     SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
     // current it is used only for read function scenarios, as file update always requires to work
@@ -169,7 +169,7 @@ public class SegmentUpdateStatusManager {
    * @return
    */
   public ICarbonLock getTableUpdateStatusLock() {
-    return CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier.getCarbonTableIdentifier(),
+    return CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier,
         LockUsage.TABLE_UPDATE_STATUS_LOCK);
   }
 
@@ -270,7 +270,7 @@ public class SegmentUpdateStatusManager {
    */
   public String[] getDeleteDeltaFilePath(String blockFilePath) throws Exception {
     int tableFactPathLength = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier()).getFactDir().length() + 1;
     String blockame = blockFilePath.substring(tableFactPathLength);
     String tupleId = CarbonTablePath.getShortBlockId(blockame);
@@ -289,7 +289,7 @@ public class SegmentUpdateStatusManager {
   public List<String> getDeltaFiles(String tupleId, String extension) throws Exception {
     try {
       CarbonTablePath carbonTablePath = CarbonStorePath
-          .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+          .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
               absoluteTableIdentifier.getCarbonTableIdentifier());
       String segment = CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID);
       String carbonDataDirectoryPath = carbonTablePath.getCarbonDataDirectoryPath("0", segment);
@@ -421,7 +421,7 @@ public class SegmentUpdateStatusManager {
   public CarbonFile[] getDeleteDeltaFilesList(final String segmentId, final String blockName) {
 
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier());
 
     String segmentPath = carbonTablePath.getCarbonDataDirectoryPath("0", segmentId);
@@ -472,7 +472,7 @@ public class SegmentUpdateStatusManager {
       CarbonFile[] allFilesOfSegment) {
 
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier());
     SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
     String endTimeStamp = "";
@@ -681,7 +681,7 @@ public class SegmentUpdateStatusManager {
     }
 
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier());
 
     String tableUpdateStatusPath =
@@ -716,7 +716,7 @@ public class SegmentUpdateStatusManager {
   private String getUpdatedStatusIdentifier() {
     SegmentStatusManager ssm = new SegmentStatusManager(absoluteTableIdentifier);
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier());
     LoadMetadataDetails[] loadDetails =
         ssm.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());
@@ -736,7 +736,7 @@ public class SegmentUpdateStatusManager {
       String updateStatusFileIdentifier) throws IOException {
 
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier());
 
     String fileLocation =
@@ -834,7 +834,7 @@ public class SegmentUpdateStatusManager {
     List<String> listOfInvalidBlocks = new ArrayList<String>();
     SegmentStatusManager ssm = new SegmentStatusManager(absoluteTableIdentifier);
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier());
     LoadMetadataDetails[] segmentDetails =
         ssm.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 51b6f06..e709df7 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -58,6 +58,7 @@ import org.apache.carbondata.core.exception.InvalidConfigurationException;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
 import org.apache.carbondata.core.keygenerator.mdkey.NumberCompressor;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
@@ -86,6 +87,7 @@ import org.apache.carbondata.format.DataChunk3;
 import com.google.gson.Gson;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.thrift.TBase;
 import org.apache.thrift.TException;
@@ -728,6 +730,28 @@ public final class CarbonUtil {
     return defaultFsUrl + currentPath;
   }
 
+  /**
+   * Append default file system schema if not added to the filepath
+   *
+   * @param filePath
+   */
+  public static String checkAndAppendFileSystemURIScheme(String filePath) {
+    String currentPath = filePath;
+
+    if (checkIfPrefixExists(filePath)) {
+      return currentPath;
+    }
+    if (!filePath.startsWith("/")) {
+      filePath = "/" + filePath;
+    }
+    currentPath = filePath;
+    String defaultFsUrl = conf.get(CarbonCommonConstants.FS_DEFAULT_FS);
+    if (defaultFsUrl == null) {
+      return currentPath;
+    }
+    return defaultFsUrl + currentPath;
+  }
+
   private static boolean checkIfPrefixExists(String path) {
     final String lowerPath = path.toLowerCase();
     return lowerPath.startsWith(CarbonCommonConstants.HDFSURL_PREFIX) || lowerPath
@@ -1006,7 +1030,7 @@ public final class CarbonUtil {
     // it will be sinkup with block index read from file
     Collections.sort(tableBlockInfoList);
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier());
     // geting the index file path
     //TODO need to pass proper partition number when partiton will be supported
@@ -1249,7 +1273,7 @@ public final class CarbonUtil {
     // it will be sinkup with block index read from file
     Collections.sort(tableBlockInfoList);
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier());
     // geting the index file path
     //TODO need to pass proper partition number when partiton will be supported
@@ -1563,13 +1587,12 @@ public final class CarbonUtil {
    *                                         tableName and columnIdentifier
    * @return
    */
-  public static boolean isFileExistsForGivenColumn(String carbonStorePath,
+  public static boolean isFileExistsForGivenColumn(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
     PathService pathService = CarbonCommonFactory.getPathService();
-    CarbonTablePath carbonTablePath = pathService.getCarbonTablePath(carbonStorePath,
-        dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier(),
-        dictionaryColumnUniqueIdentifier);
-
+    CarbonTablePath carbonTablePath = pathService
+        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
+            dictionaryColumnUniqueIdentifier);
     String dictionaryFilePath = carbonTablePath.getDictionaryFilePath(
         dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
     String dictionaryMetadataFilePath = carbonTablePath.getDictionaryMetaFilePath(
@@ -2082,5 +2105,19 @@ public final class CarbonUtil {
     return -1;
   }
 
+  /**
+   * get the parent folder of old table path and returns the new tablePath by appending new
+   * tableName to the parent
+   *
+   * @param carbonTablePath       Old tablePath
+   * @param carbonTableIdentifier new carbonTableIdentifier
+   * @return the new table path
+   */
+  public static String getNewTablePath(Path carbonTablePath,
+      CarbonTableIdentifier carbonTableIdentifier) {
+    Path parentPath = carbonTablePath.getParent();
+    return parentPath.toString() + CarbonCommonConstants.FILE_SEPARATOR + carbonTableIdentifier
+        .getTableName();
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java
index 201e05f..e57448d 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java
@@ -16,8 +16,6 @@
  */
 package org.apache.carbondata.core.util.path;
 
-import java.io.File;
-
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 
@@ -38,20 +36,13 @@ public class CarbonStorePath extends Path {
   /**
    * gets CarbonTablePath object to manage table paths
    *
-   * @param storePath the store path of the segment
+   * @param tablePath the store path of the segment
    * @param tableIdentifier identifier of carbon table that the segment belong to
    * @return the store location of the segment
    */
-  public static CarbonTablePath getCarbonTablePath(String storePath,
+  public static CarbonTablePath getCarbonTablePath(String tablePath,
       CarbonTableIdentifier tableIdentifier) {
-    return new CarbonTablePath(tableIdentifier,
-        storePath + File.separator + tableIdentifier.getDatabaseName() + File.separator
-            + tableIdentifier.getTableName());
-  }
-
-  public static CarbonTablePath getCarbonTablePath(String storePath,
-      String dbName, String tableName) {
-    return new CarbonTablePath(storePath, dbName, tableName);
+    return new CarbonTablePath(tableIdentifier, tablePath);
   }
 
   public static CarbonTablePath getCarbonTablePath(AbsoluteTableIdentifier identifier) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
index caa046f..d363ac3 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
@@ -73,19 +73,6 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * structure CarbonTablePath object to manage table paths
-   *
-   * @param storePath the store path of the segment
-   * @param dbName database name
-   * @param tableName table name
-   */
-  public CarbonTablePath(String storePath, String dbName, String tableName) {
-    super(storePath + File.separator + dbName + File.separator + tableName);
-    this.carbonTableIdentifier = new CarbonTableIdentifier(dbName, tableName, "");
-    this.tablePath = storePath + File.separator + dbName + File.separator + tableName;
-  }
-
-  /**
    * The method returns the folder path containing the carbon file.
    *
    * @param carbonFilePath

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
index ed54a3b..6e867bb 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
@@ -29,7 +29,6 @@ import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReader;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
@@ -55,11 +54,6 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
       LogServiceFactory.getLogService(CarbonDictionaryWriterImpl.class.getName());
 
   /**
-   * carbon type identifier
-   */
-  protected CarbonTableIdentifier carbonTableIdentifier;
-
-  /**
    * list which will hold values upto maximum of one dictionary chunk size
    */
   private List<ByteBuffer> oneDictionaryChunkList;
@@ -80,11 +74,6 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
   protected DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
 
   /**
-   * carbon dictionary data store path
-   */
-  protected String storePath;
-
-  /**
    * dictionary file path
    */
   protected String dictionaryFilePath;
@@ -130,15 +119,11 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
   /**
    * Constructor
    *
-   * @param storePath             carbon dictionary data store path
-   * @param carbonTableIdentifier table identifier which will give table name and database name
-   * @param dictionaryColumnUniqueIdentifier      column unique identifier
+   * @param dictionaryColumnUniqueIdentifier column unique identifier
    */
-  public CarbonDictionaryWriterImpl(String storePath, CarbonTableIdentifier carbonTableIdentifier,
+  public CarbonDictionaryWriterImpl(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    this.carbonTableIdentifier = carbonTableIdentifier;
     this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
-    this.storePath = storePath;
     this.isFirstTime = true;
   }
 
@@ -255,7 +240,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
   protected void initPaths() {
     PathService pathService = CarbonCommonFactory.getPathService();
     CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(this.storePath, carbonTableIdentifier,
+        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
             dictionaryColumnUniqueIdentifier);
     this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(
         dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
@@ -430,8 +415,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
    * @return
    */
   protected CarbonDictionaryMetadataReader getDictionaryMetadataReader() {
-    return new CarbonDictionaryMetadataReaderImpl(storePath, carbonTableIdentifier,
-        dictionaryColumnUniqueIdentifier);
+    return new CarbonDictionaryMetadataReaderImpl(dictionaryColumnUniqueIdentifier);
   }
 
   @Override public void commit() throws IOException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
index b10da11..840b25d 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
@@ -27,7 +27,6 @@ import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.service.CarbonCommonFactory;
 import org.apache.carbondata.core.service.PathService;
 import org.apache.carbondata.core.util.CarbonProperties;
@@ -43,20 +42,11 @@ import org.apache.carbondata.format.ColumnSortInfo;
 public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySortIndexWriter {
 
   /**
-   * carbonTable Identifier holding the info of databaseName and tableName
-   */
-  protected CarbonTableIdentifier carbonTableIdentifier;
-
-  /**
    * column name
    */
   protected DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
 
   /**
-   * carbon store location
-   */
-  protected String carbonStorePath;
-  /**
    * Path of dictionary sort index file for which the sortIndex to be written
    */
   protected String sortIndexFilePath;
@@ -77,16 +67,11 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
       LogServiceFactory.getLogService(CarbonDictionarySortIndexWriterImpl.class.getName());
 
   /**
-   * @param carbonStorePath       Carbon store path
-   * @param carbonTableIdentifier table identifier which will give table name and database name
    * @param dictionaryColumnUniqueIdentifier      column unique identifier
    */
-  public CarbonDictionarySortIndexWriterImpl(final CarbonTableIdentifier carbonTableIdentifier,
-      final DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier,
-      final String carbonStorePath) {
-    this.carbonTableIdentifier = carbonTableIdentifier;
+  public CarbonDictionarySortIndexWriterImpl(
+      final DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
     this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
-    this.carbonStorePath = carbonStorePath;
   }
 
   /**
@@ -151,7 +136,7 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
   protected void initPath() {
     PathService pathService = CarbonCommonFactory.getPathService();
     CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(carbonStorePath, carbonTableIdentifier,
+        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
             dictionaryColumnUniqueIdentifier);
     String dictionaryPath = carbonTablePath.getDictionaryFilePath(
         dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java b/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java
index 6d7e009..37db079 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java
@@ -65,12 +65,12 @@ public class CacheProviderTest {
     // get cache provider instance
     CacheProvider cacheProvider = CacheProvider.getInstance();
     Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictionaryCache =
-        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, "carbonStore");
+        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);
     // assert that dictionary cache is an instance of Forward dictionary cache
     assertTrue(dictionaryCache instanceof ForwardDictionaryCache);
     assertFalse(dictionaryCache instanceof ReverseDictionaryCache);
     Cache<DictionaryColumnUniqueIdentifier, Dictionary> reverseDictionaryCache =
-        cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, "carbonStore");
+        cacheProvider.createCache(CacheType.REVERSE_DICTIONARY);
     // assert that dictionary cache is an instance of Reverse dictionary cache
     assertTrue(reverseDictionaryCache instanceof ReverseDictionaryCache);
     assertFalse(reverseDictionaryCache instanceof ForwardDictionaryCache);
@@ -90,7 +90,7 @@ public class CacheProviderTest {
     CacheProvider cacheProvider = CacheProvider.getInstance();
     CarbonProperties.getInstance().addProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE, "true");
     Cache<TableSegmentUniqueIdentifier, SegmentTaskIndexStore> driverCache =
-        cacheProvider.createCache(CacheType.DRIVER_BTREE, "carbonStore");
+        cacheProvider.createCache(CacheType.DRIVER_BTREE);
     Field carbonLRUCacheField = SegmentTaskIndexStore.class.getDeclaredField("lruCache");
     carbonLRUCacheField.setAccessible(true);
     CarbonLRUCache carbonLRUCache = (CarbonLRUCache) carbonLRUCacheField.get(driverCache);
@@ -105,7 +105,7 @@ public class CacheProviderTest {
     // validation test for the executor memory.
     CarbonProperties.getInstance().addProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE, "false");
     Cache<TableBlockUniqueIdentifier, BlockIndexStore> executorCache =
-        cacheProvider.createCache(CacheType.EXECUTOR_BTREE, "carbonStore");
+        cacheProvider.createCache(CacheType.EXECUTOR_BTREE);
     carbonLRUCacheField = BlockIndexStore.class.getSuperclass().getDeclaredField("lruCache");
     carbonLRUCacheField.setAccessible(true);
     carbonLRUCache = (CarbonLRUCache) carbonLRUCacheField.get(executorCache);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
index d945f2f..54881a8 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
@@ -30,6 +30,7 @@ import java.util.List;
 import java.util.Properties;
 
 import org.apache.carbondata.core.cache.Cache;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
@@ -48,6 +49,8 @@ public class AbstractDictionaryCacheTest {
 
   protected CarbonTableIdentifier carbonTableIdentifier;
 
+  protected AbsoluteTableIdentifier absoluteTableIdentifier;
+
   protected String databaseName;
 
   protected String tableName;
@@ -105,9 +108,9 @@ public class AbstractDictionaryCacheTest {
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
       String columnId) {
 	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataTypes.STRING);
-    return new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
+    return new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
         DataTypes.STRING,
-        CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier));
+        CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
   }
 
   /**
@@ -129,13 +132,13 @@ public class AbstractDictionaryCacheTest {
       throws IOException {
 	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
             columnIdentifier.getDataType(),
-            CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier));
+            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
     CarbonDictionaryWriter carbonDictionaryWriter =
-        new CarbonDictionaryWriterImpl(carbonStorePath, carbonTableIdentifier, dictionaryColumnUniqueIdentifier);
+        new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
     CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);
+        CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     CarbonUtil.checkAndCreateFolder(carbonTablePath.getMetadataDirectoryPath());
     List<byte[]> valueList = convertStringListToByteArray(data);
     try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
index 59e348b..cea9ad7 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
@@ -24,6 +24,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -48,15 +49,17 @@ public class DictionaryCacheLoaderImplTest {
 
   @BeforeClass public static void setUp() {
     CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("db", "table1", "1");
+    AbsoluteTableIdentifier absoluteTableIdentifier = new AbsoluteTableIdentifier("/tmp",
+        carbonTableIdentifier);
     Map<String, String> columnProperties = new HashMap<>();
     columnProperties.put("prop1", "value1");
     columnProperties.put("prop2", "value2");
     columnIdentifier = new ColumnIdentifier("1", columnProperties, DataTypes.STRING);
     dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType(), CarbonStorePath.getCarbonTablePath("/tmp", carbonTableIdentifier));
-    dictionaryCacheLoader = new DictionaryCacheLoaderImpl(carbonTableIdentifier, "/tmp/",
-        dictionaryColumnUniqueIdentifier);
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
+            columnIdentifier.getDataType(), CarbonStorePath.getCarbonTablePath("/tmp",
+            carbonTableIdentifier));
+    dictionaryCacheLoader = new DictionaryCacheLoaderImpl(dictionaryColumnUniqueIdentifier);
     dictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
     new MockUp<CarbonDictionaryReaderImpl>() {
       @Mock @SuppressWarnings("unused") Iterator<byte[]> read(long startOffset, long endOffset)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
index bd8b6bc..1fb5a18 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
@@ -19,6 +19,7 @@ package org.apache.carbondata.core.cache.dictionary;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
@@ -44,17 +45,21 @@ public class DictionaryColumnUniqueIdentifierTest {
         new CarbonTableIdentifier("testDatabase", "testTable", "1");
     CarbonTableIdentifier carbonTableIdentifier2 =
         new CarbonTableIdentifier("testDatabase", "testTable", "2");
+    AbsoluteTableIdentifier absoluteTableIdentifier1 = new AbsoluteTableIdentifier("storepath",
+        carbonTableIdentifier1);
+    AbsoluteTableIdentifier absoluteTableIdentifier2 = new AbsoluteTableIdentifier("storepath",
+        carbonTableIdentifier2);
     Map<String, String> properties = new HashMap<>();
     ColumnIdentifier columnIdentifier = new ColumnIdentifier("2", properties, DataTypes.STRING);
     ColumnIdentifier columnIdentifier2 = new ColumnIdentifier("1", properties, DataTypes.INT);
     dictionaryColumnUniqueIdentifier1 =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier1, columnIdentifier,
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier1, columnIdentifier,
             DataTypes.STRING, null);
     dictionaryColumnUniqueIdentifier2 =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier2, columnIdentifier2,
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier2, columnIdentifier2,
             DataTypes.STRING, null);
     dictionaryColumnUniqueIdentifier3 =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier2, columnIdentifier,
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier2, columnIdentifier,
             DataTypes.STRING, null);
   }
 
@@ -93,6 +98,6 @@ public class DictionaryColumnUniqueIdentifierTest {
         return 2;
       }
     };
-    assertEquals(dictionaryColumnUniqueIdentifier1.hashCode(), 33);
+    assertEquals(dictionaryColumnUniqueIdentifier1.hashCode(), 937100380);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
index 34aed8a..d6041b0 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
@@ -28,6 +28,7 @@ import java.util.UUID;
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.CacheProvider;
 import org.apache.carbondata.core.cache.CacheType;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
@@ -53,7 +54,11 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
     this.databaseName = props.getProperty("database", "testSchema");
     this.tableName = props.getProperty("tableName", "carbon");
     this.carbonStorePath = props.getProperty("storePath", "carbonStore");
-    carbonTableIdentifier = new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
+    carbonTableIdentifier =
+        new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
+    absoluteTableIdentifier =
+        new AbsoluteTableIdentifier(carbonStorePath + "/" + databaseName + "/" + tableName,
+            carbonTableIdentifier);
     columnIdentifiers = new String[] { "name", "place" };
     deleteStorePath();
     prepareDataSet();
@@ -62,6 +67,7 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
 
   @After public void tearDown() throws Exception {
     carbonTableIdentifier = null;
+    absoluteTableIdentifier = null;
     forwardDictionaryCache = null;
     deleteStorePath();
   }
@@ -72,7 +78,7 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
         .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
     CacheProvider cacheProvider = CacheProvider.getInstance();
     forwardDictionaryCache =
-        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, this.carbonStorePath);
+        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);
   }
 
   @Test public void get() throws Exception {
@@ -211,9 +217,9 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
   private void writeSortIndexFile(List<String> data, String columnId) throws IOException {
 	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
             columnIdentifier.getDataType(),
-            CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier));
+            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
     Map<String, Integer> dataToSurrogateKeyMap = new HashMap<>(data.size());
     int surrogateKey = 0;
     List<Integer> invertedIndexList = new ArrayList<>(data.size());
@@ -233,8 +239,7 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
       invertedIndexList.add(invertedIndexArray[i]);
     }
     CarbonDictionarySortIndexWriter dictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, dictionaryColumnUniqueIdentifier,
-            carbonStorePath);
+        new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
     try {
       dictionarySortIndexWriter.writeSortIndex(sortedIndexList);
       dictionarySortIndexWriter.writeInvertedSortIndex(invertedIndexList);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
index d3d5658..9c5b956 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
@@ -32,6 +32,7 @@ import mockit.MockUp;
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.CacheProvider;
 import org.apache.carbondata.core.cache.CacheType;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
@@ -56,7 +57,11 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
     this.databaseName = props.getProperty("database", "testSchema");
     this.tableName = props.getProperty("tableName", "carbon");
     this.carbonStorePath = props.getProperty("storePath", "carbonStore");
-    carbonTableIdentifier = new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
+    carbonTableIdentifier =
+        new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
+    absoluteTableIdentifier =
+        new AbsoluteTableIdentifier(carbonStorePath + "/" + databaseName + "/" + tableName,
+            carbonTableIdentifier);
     columnIdentifiers = new String[] { "name", "place" };
     deleteStorePath();
     prepareDataSet();
@@ -66,6 +71,7 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
   @After public void tearDown() throws Exception {
     carbonTableIdentifier = null;
     reverseDictionaryCache = null;
+    absoluteTableIdentifier = null;
     deleteStorePath();
   }
 
@@ -76,7 +82,7 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
     CacheProvider cacheProvider = CacheProvider.getInstance();
     cacheProvider.dropAllCache();
     reverseDictionaryCache =
-        cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, this.carbonStorePath);
+        cacheProvider.createCache(CacheType.REVERSE_DICTIONARY);
   }
 
   @Test public void get() throws Exception {
@@ -272,6 +278,6 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
 	      String columnId) {
 	    ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataTypes.DOUBLE);
-    return new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier);
+    return new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier);
 	  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/carbon/AbsoluteTableIdentifierTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/AbsoluteTableIdentifierTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/AbsoluteTableIdentifierTest.java
index d55de30..6d3ae0e 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/AbsoluteTableIdentifierTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/AbsoluteTableIdentifierTest.java
@@ -33,13 +33,14 @@ public class AbsoluteTableIdentifierTest {
   static AbsoluteTableIdentifier absoluteTableIdentifier4;
 
   @BeforeClass public static void setup() {
-    absoluteTableIdentifier = new AbsoluteTableIdentifier("storePath",
+    absoluteTableIdentifier = new AbsoluteTableIdentifier("storePath/databaseName/tableName",
         new CarbonTableIdentifier("databaseName", "tableName", "tableId"));
     absoluteTableIdentifier1 = new AbsoluteTableIdentifier("dummy", null);
     absoluteTableIdentifier2 = new AbsoluteTableIdentifier("dumgfhmy", null);
     absoluteTableIdentifier3 =
-        new AbsoluteTableIdentifier("duhgmmy", new CarbonTableIdentifier("dummy", "dumy", "dmy"));
-    absoluteTableIdentifier4 = new AbsoluteTableIdentifier("storePath",
+        new AbsoluteTableIdentifier("duhgmmy/dumy/dmy/",
+            new CarbonTableIdentifier("dummy", "dumy", "dmy"));
+    absoluteTableIdentifier4 = new AbsoluteTableIdentifier("storePath/databaseName/tableName",
         new CarbonTableIdentifier("databaseName", "tableName", "tableId"));
   }
 
@@ -80,7 +81,7 @@ public class AbsoluteTableIdentifierTest {
 
   @Test public void hashCodeTest() {
     int res = absoluteTableIdentifier4.hashCode();
-    int expectedResult = 804398706;
+    int expectedResult = 233212069;
     assertEquals(res, expectedResult);
   }
 
@@ -91,13 +92,13 @@ public class AbsoluteTableIdentifierTest {
 
   @Test public void fromTablePathTest() {
     AbsoluteTableIdentifier absoluteTableIdentifierTest =
-        AbsoluteTableIdentifier.fromTablePath("storePath/databaseName/tableName");
-    assert (absoluteTableIdentifierTest.getStorePath()
-        .equals(absoluteTableIdentifier4.getStorePath()));
+        AbsoluteTableIdentifier.from("storePath/databaseName/tableName", "databaseName", "tableName");
+    assert (absoluteTableIdentifierTest.getTablePath()
+        .equals(absoluteTableIdentifier4.getTablePath()));
   }
 
-  @Test(expected = IllegalArgumentException.class) public void fromTablePathWithExceptionTest() {
-    AbsoluteTableIdentifier absoluteTableIdentifierTest =
-        AbsoluteTableIdentifier.fromTablePath("storePath/databaseName");
-  }
+//  @Test(expected = IllegalArgumentException.class) public void fromTablePathWithExceptionTest() {
+//    AbsoluteTableIdentifier absoluteTableIdentifierTest =
+//        AbsoluteTableIdentifier.fromTablePath("storePath/databaseName");
+//  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/datastore/SegmentTaskIndexStoreTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/SegmentTaskIndexStoreTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/SegmentTaskIndexStoreTest.java
index 982fb50..cec70dd 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/SegmentTaskIndexStoreTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/SegmentTaskIndexStoreTest.java
@@ -60,7 +60,7 @@ public class SegmentTaskIndexStoreTest {
     CacheProvider cacheProvider = CacheProvider.getInstance();
     taskIndexStore = (SegmentTaskIndexStore) cacheProvider.
         <TableSegmentUniqueIdentifier, SegmentTaskIndexWrapper>
-            createCache(CacheType.DRIVER_BTREE, "");
+            createCache(CacheType.DRIVER_BTREE);
     tableBlockInfo = new TableBlockInfo("file", 0L, "SG100", locations, 10L,
         ColumnarFormatVersion.valueOf(version), null);
     absoluteTableIdentifier = new AbsoluteTableIdentifier("/tmp",

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFileTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFileTest.java
index 9ae5c34..96ef106 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFileTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFileTest.java
@@ -91,7 +91,7 @@ public class LocalCarbonFileTest {
 
 
         };
-        assertTrue(localCarbonFile.listFiles(carbonFileFilter) == null);
+        assertArrayEquals(localCarbonFile.listFiles(carbonFileFilter), new CarbonFile[0]);
     }
 
     @Test
@@ -337,7 +337,7 @@ public class LocalCarbonFileTest {
         };
         localCarbonFile = new LocalCarbonFile(dir);
 
-        assertTrue(localCarbonFile.listFiles(carbonFileFilter) == null);
+        assertArrayEquals(localCarbonFile.listFiles(carbonFileFilter) , new CarbonFile[0]);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java
index 34906c3..5318f3d 100644
--- a/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java
@@ -89,7 +89,7 @@ public class DictionaryClientTest {
     tableInfo.setTableUniqueName("TestTable");
     tableInfo.setDatabaseName("test");
     storePath = System.getProperty("java.io.tmpdir") + "/tmp";
-    tableInfo.setStorePath(storePath);
+    tableInfo.setTablePath(storePath + "/" + "test" + "/" + "TestTable");
     CarbonTable carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
 
     // Add the created table to metadata

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java
index 6f09897..414a350 100644
--- a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java
@@ -58,7 +58,7 @@ public class IncrementalColumnDictionaryGeneratorTest {
     tableInfo.setTableUniqueName("TestTable");
     tableInfo.setDatabaseName("test");
     String storePath = System.getProperty("java.io.tmpdir") + "/tmp";
-    tableInfo.setStorePath(storePath);
+    tableInfo.setTablePath(storePath + "/test" + "/TestTable");
     carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
     carbonDimension = new CarbonDimension(columnSchema,0,0,0,0,0);
   }
@@ -152,7 +152,7 @@ public class IncrementalColumnDictionaryGeneratorTest {
     File dictPath = new File(storePath + "/test/TestTable/Metadata/");
     System.out.print(dictPath.mkdirs());
 
-    tableInfo.setStorePath(storePath);
+    tableInfo.setTablePath(storePath + "/test" + "/TestTable");
     CarbonTable carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
 
     // Add the table to metadata

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java
index 076968c..57bd3c0 100644
--- a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java
@@ -85,7 +85,7 @@ public class ServerDictionaryGeneratorTest {
     tableInfo.setTableUniqueName("TestTable");
     tableInfo.setDatabaseName("test");
     storePath = System.getProperty("java.io.tmpdir") + "/tmp";
-    tableInfo.setStorePath(storePath);
+    tableInfo.setTablePath(storePath + "/test" + "/TestTable");
     carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
     // Add the created table to metadata
     metadata.addCarbonTable(carbonTable);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java
index 184ed3c..f535a59 100644
--- a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java
@@ -84,7 +84,7 @@ public class TableDictionaryGeneratorTest {
     tableInfo.setTableUniqueName("TestTable");
     tableInfo.setDatabaseName("test");
     storePath = System.getProperty("java.io.tmpdir") + "/tmp";
-    tableInfo.setStorePath(storePath);
+    tableInfo.setTablePath(storePath + "/test" + "/TestTable");
     carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
     // Add the created table to metadata
     metadata.addCarbonTable(carbonTable);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java
index ddb5e3a..0de160a 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java
@@ -156,7 +156,7 @@ public class CarbonMetadataTest {
     info.setLastUpdatedTime(timeStamp);
     info.setTableUniqueName("carbonTestDatabase_carbonTestTable");
     info.setFactTable(getTableSchema());
-    info.setStorePath("/test/store");
+    info.setTablePath("/test/store/carbonTestDatabase/carbonTestTable");
     return info;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
index fa5765a..8b66233 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
@@ -109,7 +109,7 @@ public class CarbonTableTest extends TestCase {
     info.setLastUpdatedTime(timeStamp);
     info.setTableUniqueName("carbonTestDatabase_carbonTestTable");
     info.setFactTable(getTableSchema());
-    info.setStorePath("testore");
+    info.setTablePath("testore");
     return info;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
index 0aed719..e9caf4a 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
@@ -148,7 +148,7 @@ public class CarbonTableWithComplexTypesTest extends TestCase {
     info.setLastUpdatedTime(timeStamp);
     info.setTableUniqueName("carbonTestDatabase_carbonTestTable");
     info.setFactTable(getTableSchema());
-    info.setStorePath("testStore");
+    info.setTablePath("testStore");
     return info;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImplTest.java b/core/src/test/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImplTest.java
index b3fbdd6..4e3bddb 100644
--- a/core/src/test/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImplTest.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.service.impl.PathFactory;
 import org.apache.carbondata.core.service.CarbonCommonFactory;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
@@ -40,21 +41,22 @@ import org.junit.Test;
 import static org.junit.Assert.assertTrue;
 
 public class CarbonDictionaryReaderImplTest {
-  private static CarbonTableIdentifier carbonTableIdentifier = null;
+  private static AbsoluteTableIdentifier absoluteTableIdentifier = null;
   private static ColumnIdentifier columnIdentifier = null;
 
   private static CarbonDictionaryReaderImpl carbonDictionaryReaderImpl = null;
 
   @BeforeClass public static void setUp() throws Exception {
     columnIdentifier = new ColumnIdentifier("1", null, null);
-    carbonTableIdentifier =
-        new CarbonTableIdentifier("dbName", "tableName", UUID.randomUUID().toString());
+    absoluteTableIdentifier =
+        new AbsoluteTableIdentifier("storePath",
+            new CarbonTableIdentifier("dbName", "tableName", UUID.randomUUID().toString()));
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
             columnIdentifier.getDataType(),
-        CarbonStorePath.getCarbonTablePath("storePath", carbonTableIdentifier));
+        CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
     carbonDictionaryReaderImpl =
-        new CarbonDictionaryReaderImpl("storePath", carbonTableIdentifier, dictionaryColumnUniqueIdentifier);
+        new CarbonDictionaryReaderImpl(dictionaryColumnUniqueIdentifier);
   }
 
   @Test public void testRead() throws Exception {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java b/core/src/test/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
index 2a4c290..2953c33 100644
--- a/core/src/test/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
@@ -63,15 +64,21 @@ public class CarbonDictionarySortIndexReaderImplTest {
     deleteStorePath();
     CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("testSchema", "carbon",
     		UUID.randomUUID().toString());
+    AbsoluteTableIdentifier absoluteTableIdentifier =
+        new AbsoluteTableIdentifier(storePath+"/testSchema/carbon", carbonTableIdentifier);
     ColumnIdentifier columnIdentifier = new ColumnIdentifier("Name", null, null);
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier = new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier, columnIdentifier.getDataType(),
-        CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier));
-    CarbonDictionaryWriter dictionaryWriter = new CarbonDictionaryWriterImpl(storePath,
-       carbonTableIdentifier, dictionaryColumnUniqueIdentifier);
-    String metaFolderPath =storePath+File.separator+carbonTableIdentifier.getDatabaseName()+File.separator+carbonTableIdentifier.getTableName()+File.separator+"Metadata";
+    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
+            columnIdentifier.getDataType(),
+            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
+    CarbonDictionaryWriter dictionaryWriter =
+        new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
+    String metaFolderPath =
+        storePath + File.separator + carbonTableIdentifier.getDatabaseName() + File.separator
+            + carbonTableIdentifier.getTableName() + File.separator + "Metadata";
     CarbonUtil.checkAndCreateFolder(metaFolderPath);
     CarbonDictionarySortIndexWriter dictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, dictionaryColumnUniqueIdentifier, storePath);
+        new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
     List<int[]> expectedData = prepareExpectedData();
     int[] data = expectedData.get(0);
     for(int i=0;i<data.length;i++) {
@@ -85,7 +92,7 @@ public class CarbonDictionarySortIndexReaderImplTest {
     dictionarySortIndexWriter.writeInvertedSortIndex(invertedSortIndex);
     dictionarySortIndexWriter.close();
     CarbonDictionarySortIndexReader dictionarySortIndexReader =
-        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, dictionaryColumnUniqueIdentifier, storePath);
+        new CarbonDictionarySortIndexReaderImpl(dictionaryColumnUniqueIdentifier);
     List<Integer> actualSortIndex = dictionarySortIndexReader.readSortIndex();
     List<Integer> actualInvertedSortIndex = dictionarySortIndexReader.readInvertedSortIndex();
     for (int i = 0; i < actualSortIndex.size(); i++) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
index 395a68c..5f7e971 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
@@ -20,6 +20,7 @@ package org.apache.carbondata.core.util.path;
 import java.io.IOException;
 import java.util.UUID;
 
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 
 import org.junit.Test;
@@ -39,7 +40,9 @@ public class CarbonFormatDirectoryStructureTest {
   @Test public void testTablePathStructure() throws IOException {
     CarbonTableIdentifier tableIdentifier = new CarbonTableIdentifier("d1", "t1", UUID.randomUUID().toString());
     CarbonStorePath carbonStorePath = new CarbonStorePath(CARBON_STORE);
-    CarbonTablePath carbonTablePath = carbonStorePath.getCarbonTablePath(tableIdentifier);
+    AbsoluteTableIdentifier absoluteTableIdentifier =
+        new AbsoluteTableIdentifier(CARBON_STORE + "/d1/t1", tableIdentifier);
+    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     assertTrue(carbonTablePath.getPath().replace("\\", "/").equals(CARBON_STORE + "/d1/t1"));
     assertTrue(carbonTablePath.getSchemaFilePath().replace("\\", "/").equals(CARBON_STORE + "/d1/t1/Metadata/schema"));
     assertTrue(carbonTablePath.getTableStatusFilePath().replace("\\", "/")