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 16:59:39 UTC

[1/7] [carbondata] Git Push Summary

Repository: carbondata
Updated Branches:
  refs/heads/master 667ee81f1 -> 1155d4d8f

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

Posted by ra...@apache.org.
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("\\", "/")


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

Posted by ra...@apache.org.
[CARBONDATA-1573] [Integration] Support Database Location Configuration while Creating Database/ Support Creation of carbon Table in the database location

Problem:
Currently carbon stores the database and tables at the fixed configured location “carbon.storeLocation”. So even though in case of federated cluster i.e. multiple independent Namespaces/ Namenodes only one Namespace/ Namenode can be used as storage for the database and tables.
The requirement is to support when create database can specify the LOCATION; the location can be viewfs path, name service path.
Using database location attribute configuration different name nodes can be used to store different databases. Having different name nodes for different databases will scale the read/write throughput.
Solution:
Setting the LOCATION attribute for a new database is a way to work with sets of files in an HDFS directory structure outside the fixed carbon store location.
In Create database command user can specify the location, the location where database and table under that database namespace could be stored.

This closes #1418


Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/1155d4d8
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/1155d4d8
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/1155d4d8

Branch: refs/heads/master
Commit: 1155d4d8f236b7fc8e81197d7966ac5aa926488f
Parents: 667ee81
Author: mohammadshahidkhan <mo...@gmail.com>
Authored: Tue Oct 10 10:50:11 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 22:29:10 2017 +0530

----------------------------------------------------------------------
 .../carbondata/core/cache/CacheProvider.java    |  21 +-
 .../dictionary/AbstractDictionaryCache.java     |  24 +-
 .../dictionary/DictionaryCacheLoaderImpl.java   |  25 +--
 .../DictionaryColumnUniqueIdentifier.java       |  26 +--
 .../dictionary/ForwardDictionaryCache.java      |   5 +-
 .../dictionary/ManageDictionaryAndBTree.java    |  54 +++--
 .../dictionary/ReverseDictionaryCache.java      |   5 +-
 .../core/constants/CarbonCommonConstants.java   |   9 +
 .../core/datastore/BlockIndexStore.java         |   2 +-
 .../core/datastore/SegmentTaskIndexStore.java   |   8 +-
 .../datastore/filesystem/LocalCarbonFile.java   |   2 +-
 .../IncrementalColumnDictionaryGenerator.java   |  33 ++-
 .../indexstore/BlockletDataMapIndexStore.java   |   8 +-
 .../blockletindex/BlockletDataMapFactory.java   |   2 +-
 .../core/locks/CarbonLockFactory.java           |  12 +-
 .../carbondata/core/locks/CarbonLockUtil.java   |  14 +-
 .../carbondata/core/locks/HdfsFileLock.java     |  28 +--
 .../carbondata/core/locks/LocalFileLock.java    |  23 +-
 .../carbondata/core/locks/ZooKeeperLocking.java |   7 +-
 .../core/memory/HeapMemoryAllocator.java        |   2 +-
 .../core/metadata/AbsoluteTableIdentifier.java  |  63 ++----
 .../ThriftWrapperSchemaConverterImpl.java       |   4 +-
 .../core/metadata/schema/table/CarbonTable.java |   4 +-
 .../core/metadata/schema/table/TableInfo.java   |  16 +-
 .../core/mutate/CarbonUpdateUtil.java           |   6 +-
 .../CarbonDictionaryMetadataReaderImpl.java     |  22 +-
 .../core/reader/CarbonDictionaryReaderImpl.java |  22 +-
 .../CarbonDictionarySortIndexReaderImpl.java    |  26 +--
 .../executor/impl/AbstractQueryExecutor.java    |   5 +-
 .../core/scan/executor/util/QueryUtil.java      |  24 +-
 .../carbondata/core/scan/filter/FilterUtil.java |  10 +-
 .../AbstractDetailQueryResultIterator.java      |   2 +-
 .../core/service/DictionaryService.java         |  28 +--
 .../carbondata/core/service/PathService.java    |   7 +-
 .../core/service/impl/DictionaryFactory.java    |  41 +---
 .../core/service/impl/PathFactory.java          |  11 +-
 .../statusmanager/SegmentStatusManager.java     |  19 +-
 .../SegmentUpdateStatusManager.java             |  20 +-
 .../apache/carbondata/core/util/CarbonUtil.java |  51 ++++-
 .../core/util/path/CarbonStorePath.java         |  15 +-
 .../core/util/path/CarbonTablePath.java         |  13 --
 .../core/writer/CarbonDictionaryWriterImpl.java |  24 +-
 .../CarbonDictionarySortIndexWriterImpl.java    |  21 +-
 .../core/cache/CacheProviderTest.java           |   8 +-
 .../dictionary/AbstractDictionaryCacheTest.java |  15 +-
 .../DictionaryCacheLoaderImplTest.java          |  11 +-
 .../DictionaryColumnUniqueIdentifierTest.java   |  13 +-
 .../dictionary/ForwardDictionaryCacheTest.java  |  17 +-
 .../dictionary/ReverseDictionaryCacheTest.java  |  12 +-
 .../carbon/AbsoluteTableIdentifierTest.java     |  23 +-
 .../datastore/SegmentTaskIndexStoreTest.java    |   2 +-
 .../filesystem/LocalCarbonFileTest.java         |   4 +-
 .../dictionary/client/DictionaryClientTest.java |   2 +-
 ...ncrementalColumnDictionaryGeneratorTest.java |   4 +-
 .../ServerDictionaryGeneratorTest.java          |   2 +-
 .../generator/TableDictionaryGeneratorTest.java |   2 +-
 .../core/metadata/CarbonMetadataTest.java       |   2 +-
 .../metadata/schema/table/CarbonTableTest.java  |   2 +-
 .../table/CarbonTableWithComplexTypesTest.java  |   2 +-
 .../reader/CarbonDictionaryReaderImplTest.java  |  14 +-
 ...CarbonDictionarySortIndexReaderImplTest.java |  21 +-
 .../CarbonFormatDirectoryStructureTest.java     |   5 +-
 .../writer/CarbonDictionaryWriterImplTest.java  |  15 +-
 ...CarbonDictionarySortIndexWriterImplTest.java |  18 +-
 .../apache/carbondata/hadoop/CacheClient.java   |   4 +-
 .../carbondata/hadoop/CarbonInputFormat.java    |  36 ++-
 .../hadoop/api/CarbonTableInputFormat.java      |  46 +++-
 .../hadoop/api/DistributableDataMapFormat.java  |   2 -
 .../internal/index/impl/InMemoryBTreeIndex.java |   2 +-
 .../impl/DictionaryDecodeReadSupport.java       |   4 +-
 .../streaming/CarbonStreamInputFormat.java      |   2 +-
 .../streaming/CarbonStreamRecordReader.java     |   2 +-
 .../hadoop/util/CarbonInputFormatUtil.java      |   8 +
 .../carbondata/hadoop/util/SchemaReader.java    |   2 +-
 .../hadoop/ft/CarbonInputMapperTest.java        |   4 +
 .../hadoop/test/util/StoreCreator.java          |  41 ++--
 .../hive/CarbonDictionaryDecodeReadSupport.java |   8 +-
 .../hive/MapredCarbonInputFormat.java           |   4 +-
 .../carbondata/hiveexample/HiveExample.scala    |   2 +-
 .../presto/CarbondataRecordSetProvider.java     |   7 +-
 .../presto/impl/CarbonTableReader.java          |  19 +-
 .../CarbonDictionaryDecodeReadSupport.scala     |   6 +-
 .../presto/util/CarbonDataStoreCreator.scala    |  32 ++-
 .../CarbonV1toV3CompatabilityTestCase.scala     |   2 +-
 .../sdv/generated/MergeIndexTestCase.scala      |  14 +-
 .../src/test/resources/dblocation/test.csv      |   6 +
 .../dataload/TestLoadDataGeneral.scala          |   4 +-
 .../InsertIntoCarbonTableTestCase.scala         |  14 +-
 ...ompactionSupportGlobalSortFunctionTest.scala |   5 +-
 ...mpactionSupportGlobalSortParameterTest.scala |   7 +-
 .../datacompaction/DataCompactionLockTest.scala |   6 +-
 .../MajorCompactionIgnoreInMinorTest.scala      |  54 +++--
 .../MajorCompactionStopsAfterCompaction.scala   |  25 +--
 .../dataload/TestBatchSortDataLoad.scala        |  10 +-
 .../dataload/TestDataLoadWithFileName.scala     |   9 +-
 .../dataload/TestGlobalSortDataLoad.scala       |  15 +-
 .../testsuite/datamap/DataMapWriterSuite.scala  | 117 +++++-----
 .../dataretention/DataRetentionTestCase.scala   |  46 ++--
 .../DBLocationCarbonTableTestCase.scala         | 223 +++++++++++++++++++
 .../TestDataLoadingForPartitionTable.scala      |   4 +-
 .../org/apache/carbondata/api/CarbonStore.scala |  11 +-
 .../spark/rdd/AlterTableAddColumnRDD.scala      |  10 +-
 .../spark/rdd/AlterTableDropColumnRDD.scala     |   8 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |  21 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |   2 +-
 .../spark/rdd/CarbonScanPartitionRDD.scala      |  21 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |  11 +-
 .../apache/carbondata/spark/rdd/Compactor.scala |   8 +-
 .../spark/rdd/DataManagementFunc.scala          |  21 +-
 .../carbondata/spark/rdd/PartitionDropper.scala |  11 +-
 .../spark/rdd/PartitionSplitter.scala           |  11 +-
 .../spark/tasks/DictionaryWriterTask.scala      |  11 +-
 .../spark/tasks/SortIndexWriterTask.scala       |   7 +-
 .../carbondata/spark/util/CommonUtil.scala      |  26 ++-
 .../carbondata/spark/util/DataLoadingUtil.scala |   2 +-
 .../spark/util/GlobalDictionaryUtil.scala       |  41 ++--
 .../command/carbonTableSchemaCommon.scala       |   6 +-
 .../spark/sql/test/TestQueryExecutor.scala      |   1 +
 .../apache/spark/sql/test/util/QueryTest.scala  |   1 +
 .../org/apache/spark/util/PartitionUtils.scala  |   7 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  22 +-
 .../carbondata/spark/util/CarbonSparkUtil.scala |   5 +-
 .../spark/sql/CarbonCatalystOperators.scala     |  37 +++
 .../sql/CarbonDatasourceHadoopRelation.scala    |   3 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |  42 ++--
 .../org/apache/spark/sql/CarbonSource.scala     |  22 +-
 .../command/CarbonCreateTableCommand.scala      |  12 +-
 .../command/CarbonDropTableCommand.scala        |  21 +-
 .../datamap/CarbonDropDataMapCommand.scala      |  17 +-
 .../AlterTableCompactionCommand.scala           |   6 +-
 .../command/management/CleanFilesCommand.scala  |   8 +-
 .../command/management/LoadTableCommand.scala   |  24 +-
 .../command/mutation/DeleteExecution.scala      |  12 +-
 .../mutation/ProjectForDeleteCommand.scala      |   2 +-
 .../mutation/ProjectForUpdateCommand.scala      |   6 +-
 .../AlterTableDropCarbonPartitionCommand.scala  |  16 +-
 .../AlterTableSplitCarbonPartitionCommand.scala |  18 +-
 .../preaaggregate/PreAggregateUtil.scala        |  15 +-
 .../CarbonAlterTableAddColumnCommand.scala      |  14 +-
 .../CarbonAlterTableDataTypeChangeCommand.scala |   4 +-
 .../CarbonAlterTableDropColumnCommand.scala     |   7 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |  36 +--
 .../strategy/CarbonLateDecodeStrategy.scala     |   1 -
 .../sql/execution/strategy/DDLStrategy.scala    |   2 -
 .../spark/sql/hive/CarbonFileMetastore.scala    | 134 +++++------
 .../spark/sql/hive/CarbonHiveMetaStore.scala    |  66 +++---
 .../apache/spark/sql/hive/CarbonMetaStore.scala |  17 +-
 .../spark/sql/hive/CarbonSessionState.scala     |   2 +-
 .../execution/command/CarbonHiveCommands.scala  |  17 +-
 .../org/apache/spark/util/AlterTableUtil.scala  |  49 ++--
 .../org/apache/spark/util/CleanFiles.scala      |   2 +-
 .../org/apache/spark/util/Compaction.scala      |   2 +-
 .../apache/spark/util/DeleteSegmentByDate.scala |   2 +-
 .../apache/spark/util/DeleteSegmentById.scala   |   2 +-
 .../org/apache/spark/util/TableLoader.scala     |   2 +-
 .../partition/TestAlterPartitionTable.scala     |   4 +-
 .../spark/util/AllDictionaryTestCase.scala      |   2 +-
 .../spark/util/DictionaryTestCaseUtil.scala     |   7 +-
 .../util/ExternalColumnDictionaryTestCase.scala |   2 +-
 .../TestStreamingTableOperation.scala           |  15 +-
 .../AlterTableValidationTestCase.scala          |   2 +-
 .../apache/spark/util/CarbonCommandSuite.scala  |   6 +-
 .../processing/datatypes/PrimitiveDataType.java |  15 +-
 .../loading/DataLoadProcessBuilder.java         |   2 +-
 .../impl/DictionaryFieldConverterImpl.java      |  11 +-
 .../converter/impl/FieldEncoderFactory.java     |  59 ++---
 .../converter/impl/RowConverterImpl.java        |  16 +-
 .../loading/model/CarbonLoadModel.java          |  18 +-
 .../merger/AbstractResultProcessor.java         |   2 +-
 .../processing/merger/CarbonDataMergerUtil.java |  32 ++-
 .../store/CarbonFactDataHandlerModel.java       |  22 +-
 .../processing/util/CarbonLoaderUtil.java       |  29 ++-
 .../processing/util/DeleteLoadFolders.java      |  19 +-
 .../carbon/datastore/BlockIndexStoreTest.java   |   6 +-
 .../carbondata/lcm/locks/LocalFileLockTest.java |   9 +-
 .../lcm/locks/ZooKeeperLockingTest.java         |   7 +-
 .../carbondata/processing/StoreCreator.java     |  44 ++--
 .../CarbonStreamingQueryListener.scala          |   2 +-
 178 files changed, 1598 insertions(+), 1367 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java b/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
index 5c4b265..c3f8540 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
@@ -84,12 +84,11 @@ public class CacheProvider {
    * it is not present in the map
    *
    * @param cacheType       type of cache
-   * @param carbonStorePath store path
    * @param <K>
    * @param <V>
    * @return
    */
-  public <K, V> Cache<K, V> createCache(CacheType cacheType, String carbonStorePath) {
+  public <K, V> Cache<K, V> createCache(CacheType cacheType) {
     //check if lru cache is null, if null create one
     //check if cache is null for given cache type, if null create one
     if (!dictionaryCacheAlreadyExists(cacheType)) {
@@ -98,7 +97,7 @@ public class CacheProvider {
           if (null == carbonLRUCache) {
             createLRULevelCacheInstance(cacheType);
           }
-          createDictionaryCacheForGivenType(cacheType, carbonStorePath);
+          createDictionaryCacheForGivenType(cacheType);
         }
       }
     }
@@ -109,26 +108,22 @@ public class CacheProvider {
    * This method will create the cache for given cache type
    *
    * @param cacheType       type of cache
-   * @param carbonStorePath store path
    */
-  private void createDictionaryCacheForGivenType(CacheType cacheType, String carbonStorePath) {
+  private void createDictionaryCacheForGivenType(CacheType cacheType) {
     Cache cacheObject = null;
     if (cacheType.equals(CacheType.REVERSE_DICTIONARY)) {
       cacheObject =
-          new ReverseDictionaryCache<DictionaryColumnUniqueIdentifier, Dictionary>(carbonStorePath,
-              carbonLRUCache);
+          new ReverseDictionaryCache<DictionaryColumnUniqueIdentifier, Dictionary>(carbonLRUCache);
     } else if (cacheType.equals(CacheType.FORWARD_DICTIONARY)) {
       cacheObject =
-          new ForwardDictionaryCache<DictionaryColumnUniqueIdentifier, Dictionary>(carbonStorePath,
-              carbonLRUCache);
+          new ForwardDictionaryCache<DictionaryColumnUniqueIdentifier, Dictionary>(carbonLRUCache);
     } else if (cacheType.equals(cacheType.EXECUTOR_BTREE)) {
-      cacheObject = new BlockIndexStore<TableBlockUniqueIdentifier, AbstractIndex>(carbonStorePath,
-          carbonLRUCache);
+      cacheObject = new BlockIndexStore<TableBlockUniqueIdentifier, AbstractIndex>(carbonLRUCache);
     } else if (cacheType.equals(cacheType.DRIVER_BTREE)) {
       cacheObject =
-          new SegmentTaskIndexStore(carbonStorePath, carbonLRUCache);
+          new SegmentTaskIndexStore(carbonLRUCache);
     } else if (cacheType.equals(cacheType.DRIVER_BLOCKLET_DATAMAP)) {
-      cacheObject = new BlockletDataMapIndexStore(carbonStorePath, carbonLRUCache);
+      cacheObject = new BlockletDataMapIndexStore(carbonLRUCache);
     }
     cacheTypeToCacheMap.put(cacheType, cacheObject);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
index e145cb8..d527e68 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
@@ -52,17 +52,11 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
    */
   protected CarbonLRUCache carbonLRUCache;
 
-  /**
-   * c store path
-   */
-  protected String carbonStorePath;
 
   /**
-   * @param carbonStorePath
    * @param carbonLRUCache
    */
-  public AbstractDictionaryCache(String carbonStorePath, CarbonLRUCache carbonLRUCache) {
-    this.carbonStorePath = carbonStorePath;
+  public AbstractDictionaryCache(CarbonLRUCache carbonLRUCache) {
     this.carbonLRUCache = carbonLRUCache;
     initThreadPoolSize();
   }
@@ -86,8 +80,7 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) throws IOException {
     DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
     CarbonDictionaryMetadataReader columnMetadataReaderImpl = dictService
-        .getDictionaryMetadataReader(dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier(),
-            dictionaryColumnUniqueIdentifier, carbonStorePath);
+        .getDictionaryMetadataReader(dictionaryColumnUniqueIdentifier);
 
     CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk = null;
     // read metadata file
@@ -113,9 +106,7 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
           throws IOException {
     DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
     CarbonDictionaryMetadataReader columnMetadataReaderImpl = dictService
-            .getDictionaryMetadataReader(
-                    dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier(),
-                    dictionaryColumnUniqueIdentifier, carbonStorePath);
+            .getDictionaryMetadataReader(dictionaryColumnUniqueIdentifier);
 
     CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk = null;
     // read metadata file
@@ -151,9 +142,9 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
   private CarbonFile getDictionaryMetaCarbonFile(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) throws IOException {
     PathService pathService = CarbonCommonFactory.getPathService();
-    CarbonTablePath carbonTablePath = pathService.getCarbonTablePath(carbonStorePath,
-        dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier(),
-        dictionaryColumnUniqueIdentifier);
+    CarbonTablePath carbonTablePath = pathService
+        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
+            dictionaryColumnUniqueIdentifier);
     String dictionaryFilePath = carbonTablePath.getDictionaryMetaFilePath(
         dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
     FileFactory.FileType fileType = FileFactory.getFileType(dictionaryFilePath);
@@ -274,8 +265,7 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
       long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
       throws IOException {
     DictionaryCacheLoader dictionaryCacheLoader =
-        new DictionaryCacheLoaderImpl(dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier(),
-            carbonStorePath, dictionaryColumnUniqueIdentifier);
+        new DictionaryCacheLoaderImpl(dictionaryColumnUniqueIdentifier);
     dictionaryCacheLoader
         .load(dictionaryInfo, dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
             dictionaryChunkStartOffset, dictionaryChunkEndOffset, loadSortIndex);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
index e6a4022..a603c01 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
@@ -23,7 +23,6 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.reader.CarbonDictionaryReader;
 import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
@@ -37,25 +36,15 @@ import org.apache.carbondata.core.util.CarbonUtil;
 public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
 
   /**
-   * carbon table identifier
+   * carbon dictionary column identifier
    */
-  private CarbonTableIdentifier carbonTableIdentifier;
-
   private DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
 
   /**
-   * carbon store path
+   * @param dictionaryColumnUniqueIdentifier dictionary column identifier
    */
-  private String carbonStorePath;
-
-  /**
-   * @param carbonTableIdentifier fully qualified table name
-   * @param carbonStorePath       hdfs store path
-   */
-  public DictionaryCacheLoaderImpl(CarbonTableIdentifier carbonTableIdentifier,
-      String carbonStorePath, DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    this.carbonTableIdentifier = carbonTableIdentifier;
-    this.carbonStorePath = carbonStorePath;
+  public DictionaryCacheLoaderImpl(
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
     this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
   }
 
@@ -166,8 +155,7 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
   private CarbonDictionaryReader getDictionaryReader(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
     DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
-    return dictService.getDictionaryReader(carbonTableIdentifier, dictionaryColumnUniqueIdentifier,
-        carbonStorePath);
+    return dictService.getDictionaryReader(dictionaryColumnUniqueIdentifier);
   }
 
   /**
@@ -178,7 +166,6 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
     DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
     return dictService
-        .getDictionarySortIndexReader(carbonTableIdentifier, dictionaryColumnUniqueIdentifier,
-            carbonStorePath);
+        .getDictionarySortIndexReader(dictionaryColumnUniqueIdentifier);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
index 3effe8a..1c733b4 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
@@ -19,7 +19,7 @@ package org.apache.carbondata.core.cache.dictionary;
 
 import java.io.Serializable;
 
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -34,7 +34,7 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
   /**
    * table fully qualified name
    */
-  private CarbonTableIdentifier carbonTableIdentifier;
+  private AbsoluteTableIdentifier absoluteTableIdentifier;
 
   /**
    * unique column id
@@ -49,18 +49,18 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
    * Will be used in case of reverse dictionary cache which will be used
    * in case of data loading.
    *
-   * @param carbonTableIdentifier
+   * @param absoluteTableIdentifier
    * @param columnIdentifier
    */
-  public DictionaryColumnUniqueIdentifier(CarbonTableIdentifier carbonTableIdentifier,
+  public DictionaryColumnUniqueIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier,
       ColumnIdentifier columnIdentifier) {
-    if (carbonTableIdentifier == null) {
+    if (absoluteTableIdentifier == null) {
       throw new IllegalArgumentException("carbonTableIdentifier is null");
     }
     if (columnIdentifier == null) {
       throw new IllegalArgumentException("columnIdentifier is null");
     }
-    this.carbonTableIdentifier = carbonTableIdentifier;
+    this.absoluteTableIdentifier = absoluteTableIdentifier;
     this.columnIdentifier = columnIdentifier;
     this.dataType = columnIdentifier.getDataType();
   }
@@ -69,13 +69,13 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
    * Will be used in case of forward dictionary cache in case
    * of query execution.
    *
-   * @param carbonTableIdentifier
+   * @param absoluteTableIdentifier
    * @param columnIdentifier
    * @param dataType
    */
-  public DictionaryColumnUniqueIdentifier(CarbonTableIdentifier carbonTableIdentifier,
+  public DictionaryColumnUniqueIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier,
       ColumnIdentifier columnIdentifier, DataType dataType, CarbonTablePath carbonTablePath) {
-    this(carbonTableIdentifier, columnIdentifier);
+    this(absoluteTableIdentifier, columnIdentifier);
     this.dataType = dataType;
     if (null != carbonTablePath) {
       this.carbonTablePath = carbonTablePath;
@@ -89,8 +89,8 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
   /**
    * @return table identifier
    */
-  public CarbonTableIdentifier getCarbonTableIdentifier() {
-    return carbonTableIdentifier;
+  public AbsoluteTableIdentifier getAbsoluteCarbonTableIdentifier() {
+    return absoluteTableIdentifier;
   }
 
   public CarbonTablePath getCarbonTablePath() {
@@ -114,7 +114,7 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
     if (this == other) return true;
     if (other == null || getClass() != other.getClass()) return false;
     DictionaryColumnUniqueIdentifier that = (DictionaryColumnUniqueIdentifier) other;
-    if (!carbonTableIdentifier.equals(that.carbonTableIdentifier)) return false;
+    if (!absoluteTableIdentifier.equals(that.absoluteTableIdentifier)) return false;
     return columnIdentifier.equals(that.columnIdentifier);
 
   }
@@ -125,7 +125,7 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
    * @return
    */
   @Override public int hashCode() {
-    int result = carbonTableIdentifier.hashCode();
+    int result = absoluteTableIdentifier.hashCode();
     result = 31 * result + columnIdentifier.hashCode();
     return result;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
index 6500bb9..f43e21b 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
@@ -59,11 +59,10 @@ public class ForwardDictionaryCache<K extends
   private static final long byteArraySize = ObjectSizeCalculator.estimate(new byte[0], 16);
 
   /**
-   * @param carbonStorePath
    * @param carbonLRUCache
    */
-  public ForwardDictionaryCache(String carbonStorePath, CarbonLRUCache carbonLRUCache) {
-    super(carbonStorePath, carbonLRUCache);
+  public ForwardDictionaryCache(CarbonLRUCache carbonLRUCache) {
+    super(carbonLRUCache);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
index bae9189..a6c89e0 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
@@ -30,7 +30,6 @@ import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 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.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -58,12 +57,11 @@ public class ManageDictionaryAndBTree {
    *
    * @param columnSchema
    * @param carbonTableIdentifier
-   * @param storePath
    */
   public static void deleteDictionaryFileAndCache(final ColumnSchema columnSchema,
-      CarbonTableIdentifier carbonTableIdentifier, String storePath) {
+      AbsoluteTableIdentifier carbonTableIdentifier) {
     CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier);
+        CarbonStorePath.getCarbonTablePath(carbonTableIdentifier);
     String metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath();
     CarbonFile metadataDir = FileFactory
         .getCarbonFile(metadataDirectoryPath, FileFactory.getFileType(metadataDirectoryPath));
@@ -92,27 +90,7 @@ public class ManageDictionaryAndBTree {
       }
     }
     // remove dictionary cache
-    removeDictionaryColumnFromCache(carbonTableIdentifier, storePath,
-        columnSchema.getColumnUniqueId());
-  }
-
-  /**
-   * This method will remove dictionary cache from driver for both reverse and forward dictionary
-   *
-   * @param carbonTableIdentifier
-   * @param storePath
-   * @param columnId
-   */
-  public static void removeDictionaryColumnFromCache(CarbonTableIdentifier carbonTableIdentifier,
-      String storePath, String columnId) {
-    Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictCache =
-        CacheProvider.getInstance().createCache(CacheType.REVERSE_DICTIONARY, storePath);
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
-            new ColumnIdentifier(columnId, null, null));
-    dictCache.invalidate(dictionaryColumnUniqueIdentifier);
-    dictCache = CacheProvider.getInstance().createCache(CacheType.FORWARD_DICTIONARY, storePath);
-    dictCache.invalidate(dictionaryColumnUniqueIdentifier);
+    removeDictionaryColumnFromCache(carbonTableIdentifier, columnSchema.getColumnUniqueId());
   }
 
   /**
@@ -136,12 +114,30 @@ public class ManageDictionaryAndBTree {
     List<CarbonDimension> dimensions =
         carbonTable.getDimensionByTableName(carbonTable.getFactTableName());
     for (CarbonDimension dimension : dimensions) {
-      removeDictionaryColumnFromCache(carbonTable.getCarbonTableIdentifier(),
-          carbonTable.getStorePath(), dimension.getColumnId());
+      removeDictionaryColumnFromCache(carbonTable.getAbsoluteTableIdentifier(),
+          dimension.getColumnId());
     }
   }
 
   /**
+   * This method will remove dictionary cache from driver for both reverse and forward dictionary
+   *
+   * @param carbonTableIdentifier
+   * @param columnId
+   */
+  public static void removeDictionaryColumnFromCache(AbsoluteTableIdentifier carbonTableIdentifier,
+      String columnId) {
+    Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictCache =
+        CacheProvider.getInstance().createCache(CacheType.REVERSE_DICTIONARY);
+    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
+        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
+            new ColumnIdentifier(columnId, null, null));
+    dictCache.invalidate(dictionaryColumnUniqueIdentifier);
+    dictCache = CacheProvider.getInstance().createCache(CacheType.FORWARD_DICTIONARY);
+    dictCache.invalidate(dictionaryColumnUniqueIdentifier);
+  }
+
+  /**
    * This method will remove the BTree instances from LRU cache
    *
    * @param absoluteTableIdentifier
@@ -149,8 +145,8 @@ public class ManageDictionaryAndBTree {
    */
   public static void invalidateBTreeCache(AbsoluteTableIdentifier absoluteTableIdentifier,
       String[] segments) {
-    Cache<Object, Object> driverBTreeCache = CacheProvider.getInstance()
-        .createCache(CacheType.DRIVER_BTREE, absoluteTableIdentifier.getStorePath());
+    Cache<Object, Object> driverBTreeCache =
+        CacheProvider.getInstance().createCache(CacheType.DRIVER_BTREE);
     for (String segmentNo : segments) {
       TableSegmentUniqueIdentifier tableSegmentUniqueIdentifier =
           new TableSegmentUniqueIdentifier(absoluteTableIdentifier, segmentNo);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCache.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
index fd5a954..05de1d3 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
@@ -64,11 +64,10 @@ public class ReverseDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
 
 
   /**
-   * @param carbonStorePath
    * @param carbonLRUCache
    */
-  public ReverseDictionaryCache(String carbonStorePath, CarbonLRUCache carbonLRUCache) {
-    super(carbonStorePath, carbonLRUCache);
+  public ReverseDictionaryCache(CarbonLRUCache carbonLRUCache) {
+    super(carbonLRUCache);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 5f24381..aeca19f 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -1345,6 +1345,15 @@ public final class CarbonCommonConstants {
   public static final String CARBON_USE_BLOCKLET_DISTRIBUTION = "carbon.blocklet.distribution";
 
   public static final String CARBON_USE_BLOCKLET_DISTRIBUTION_DEFAULT = "true";
+  /**
+   * The property to configure the mdt file folder path, earlier it was pointing to the
+   * fixed carbon store path. This is needed in case of the federation setup when user removes
+   * the fixedtorepath namesevice
+   */
+  @CarbonProperty
+  public static final String CARBON_UPDATE_SYNC_FOLDER = "carbon.update.sync.folder";
+
+  public static final String CARBON_UPDATE_SYNC_FOLDER_DEFAULT = "/tmp/carbondata";
 
   /**
    * this will be used to pass bitset value in filter to another filter for

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/datastore/BlockIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/BlockIndexStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/BlockIndexStore.java
index c9c5b3d..f2c38fa 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/BlockIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/BlockIndexStore.java
@@ -59,7 +59,7 @@ public class BlockIndexStore<K, V> extends AbstractBlockIndexStoreCache<K, V> {
    */
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(BlockIndexStore.class.getName());
-  public BlockIndexStore(String carbonStorePath, CarbonLRUCache lruCache) {
+  public BlockIndexStore(CarbonLRUCache lruCache) {
     super(lruCache);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/datastore/SegmentTaskIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/SegmentTaskIndexStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/SegmentTaskIndexStore.java
index 980ea3e..8ed5c18 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/SegmentTaskIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/SegmentTaskIndexStore.java
@@ -56,10 +56,6 @@ public class SegmentTaskIndexStore
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(SegmentTaskIndexStore.class.getName());
   /**
-   * carbon store path
-   */
-  protected String carbonStorePath;
-  /**
    * CarbonLRU cache
    */
   protected CarbonLRUCache lruCache;
@@ -78,11 +74,9 @@ public class SegmentTaskIndexStore
   /**
    * constructor to initialize the SegmentTaskIndexStore
    *
-   * @param carbonStorePath
    * @param lruCache
    */
-  public SegmentTaskIndexStore(String carbonStorePath, CarbonLRUCache lruCache) {
-    this.carbonStorePath = carbonStorePath;
+  public SegmentTaskIndexStore(CarbonLRUCache lruCache) {
     this.lruCache = lruCache;
     segmentLockMap = new ConcurrentHashMap<String, Object>();
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFile.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFile.java b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFile.java
index 89a5cd3..f6e9f8f 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFile.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFile.java
@@ -52,7 +52,7 @@ public class LocalCarbonFile implements CarbonFile {
 
   @Override public CarbonFile[] listFiles(final CarbonFileFilter fileFilter) {
     if (!file.isDirectory()) {
-      return null;
+      return new CarbonFile[0];
     }
 
     File[] files = file.listFiles(new FileFilter() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
index f361d06..a129671 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
@@ -33,7 +33,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.devapi.BiDictionary;
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.devapi.DictionaryGenerator;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -118,35 +118,33 @@ public class IncrementalColumnDictionaryGenerator implements BiDictionary<Intege
     // initialize params
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
-    CarbonTableIdentifier tableIdentifier = carbonTable.getCarbonTableIdentifier();
+    AbsoluteTableIdentifier absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier();
     ColumnIdentifier columnIdentifier = dimension.getColumnIdentifier();
-    String storePath = carbonTable.getStorePath();
     DictionaryService dictionaryService = CarbonCommonFactory.getDictionaryService();
     // create dictionary cache from dictionary File
     DictionaryColumnUniqueIdentifier identifier =
-            new DictionaryColumnUniqueIdentifier(tableIdentifier, columnIdentifier,
+            new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
                     columnIdentifier.getDataType(), carbonTablePath);
-    Boolean isDictExists = CarbonUtil.isFileExistsForGivenColumn(storePath, identifier);
+    Boolean isDictExists = CarbonUtil.isFileExistsForGivenColumn(identifier);
     Dictionary dictionary = null;
     long t1 = System.currentTimeMillis();
     if (isDictExists) {
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictCache = CacheProvider.getInstance()
-              .createCache(CacheType.REVERSE_DICTIONARY, storePath);
+              .createCache(CacheType.REVERSE_DICTIONARY);
       dictionary = dictCache.get(identifier);
     }
     long dictCacheTime = System.currentTimeMillis() - t1;
     long t2 = System.currentTimeMillis();
     // write dictionary
     CarbonDictionaryWriter dictionaryWriter = null;
-    dictionaryWriter = dictionaryService
-            .getDictionaryWriter(tableIdentifier, identifier, storePath);
+    dictionaryWriter = dictionaryService.getDictionaryWriter(identifier);
     List<String> distinctValues = writeDictionary(dictionaryWriter, isDictExists);
     long dictWriteTime = System.currentTimeMillis() - t2;
     long t3 = System.currentTimeMillis();
     // write sort index
     if (distinctValues.size() > 0) {
       writeSortIndex(distinctValues, dictionary,
-              dictionaryService, tableIdentifier, columnIdentifier, storePath);
+              dictionaryService, absoluteTableIdentifier, columnIdentifier);
     }
     long sortIndexWriteTime = System.currentTimeMillis() - t3;
     // update Meta Data
@@ -208,23 +206,19 @@ public class IncrementalColumnDictionaryGenerator implements BiDictionary<Intege
    * @param distinctValues
    * @param dictionary
    * @param dictionaryService
-   * @param tableIdentifier
+   * @param absoluteTableIdentifier
    * @param columnIdentifier
-   * @param storePath
    * @throws IOException
    */
   private void writeSortIndex(List<String> distinctValues,
                               Dictionary dictionary,
                               DictionaryService dictionaryService,
-                              CarbonTableIdentifier tableIdentifier,
-                              ColumnIdentifier columnIdentifier,
-                              String storePath) throws IOException {
+                              AbsoluteTableIdentifier absoluteTableIdentifier,
+                              ColumnIdentifier columnIdentifier) throws IOException {
     CarbonDictionarySortIndexWriter carbonDictionarySortIndexWriter = null;
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(storePath, tableIdentifier.getDatabaseName(),
-            tableIdentifier.getTableName());
+    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(tableIdentifier, columnIdentifier,
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
             columnIdentifier.getDataType(), carbonTablePath);
     try {
       CarbonDictionarySortInfoPreparator preparator = new CarbonDictionarySortInfoPreparator();
@@ -232,8 +226,7 @@ public class IncrementalColumnDictionaryGenerator implements BiDictionary<Intege
               preparator.getDictionarySortInfo(distinctValues, dictionary,
                       dimension.getDataType());
       carbonDictionarySortIndexWriter = dictionaryService
-          .getDictionarySortIndexWriter(tableIdentifier, dictionaryColumnUniqueIdentifier,
-              storePath);
+          .getDictionarySortIndexWriter(dictionaryColumnUniqueIdentifier);
       carbonDictionarySortIndexWriter.writeSortIndex(dictionarySortInfo.getSortIndex());
       carbonDictionarySortIndexWriter
               .writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
index 1ea4806..4d90a08 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
@@ -42,10 +42,6 @@ public class BlockletDataMapIndexStore
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(BlockletDataMapIndexStore.class.getName());
   /**
-   * carbon store path
-   */
-  protected String carbonStorePath;
-  /**
    * CarbonLRU cache
    */
   protected CarbonLRUCache lruCache;
@@ -61,11 +57,9 @@ public class BlockletDataMapIndexStore
   /**
    * constructor to initialize the SegmentTaskIndexStore
    *
-   * @param carbonStorePath
    * @param lruCache
    */
-  public BlockletDataMapIndexStore(String carbonStorePath, CarbonLRUCache lruCache) {
-    this.carbonStorePath = carbonStorePath;
+  public BlockletDataMapIndexStore(CarbonLRUCache lruCache) {
     this.lruCache = lruCache;
     segmentLockMap = new ConcurrentHashMap<String, Object>();
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
index e58c911..2e2cab5 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
@@ -61,7 +61,7 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
   public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
     this.identifier = identifier;
     cache = CacheProvider.getInstance()
-        .createCache(CacheType.DRIVER_BLOCKLET_DATAMAP, identifier.getStorePath());
+        .createCache(CacheType.DRIVER_BLOCKLET_DATAMAP);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
index 7480208..e70e655 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
@@ -20,7 +20,7 @@ package org.apache.carbondata.core.locks;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.util.CarbonProperties;
 
 /**
@@ -46,21 +46,21 @@ public class CarbonLockFactory {
   /**
    * This method will determine the lock type.
    *
-   * @param tableIdentifier
+   * @param absoluteTableIdentifier
    * @param lockFile
    * @return
    */
-  public static ICarbonLock getCarbonLockObj(CarbonTableIdentifier tableIdentifier,
+  public static ICarbonLock getCarbonLockObj(AbsoluteTableIdentifier absoluteTableIdentifier,
       String lockFile) {
     switch (lockTypeConfigured) {
       case CarbonCommonConstants.CARBON_LOCK_TYPE_LOCAL:
-        return new LocalFileLock(tableIdentifier, lockFile);
+        return new LocalFileLock(absoluteTableIdentifier, lockFile);
 
       case CarbonCommonConstants.CARBON_LOCK_TYPE_ZOOKEEPER:
-        return new ZooKeeperLocking(tableIdentifier, lockFile);
+        return new ZooKeeperLocking(absoluteTableIdentifier, lockFile);
 
       case CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS:
-        return new HdfsFileLock(tableIdentifier, lockFile);
+        return new HdfsFileLock(absoluteTableIdentifier, lockFile);
 
       default:
         throw new UnsupportedOperationException("Not supported the lock type");

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
index c02a168..711390b 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
@@ -19,7 +19,7 @@ package org.apache.carbondata.core.locks;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 
 /**
  * This class contains all carbon lock utilities
@@ -65,10 +65,14 @@ public class CarbonLockUtil {
   /**
    * Given a lock type this method will return a new lock object if not acquired by any other
    * operation
+   *
+   * @param absoluteTableIdentifier
+   * @param lockType
+   * @return
    */
-  public static ICarbonLock getLockObject(CarbonTableIdentifier identifier, String lockType,
-      String errorMsg) {
-    ICarbonLock carbonLock = CarbonLockFactory.getCarbonLockObj(identifier, lockType);
+  public static ICarbonLock getLockObject(AbsoluteTableIdentifier absoluteTableIdentifier,
+      String lockType, String errorMsg) {
+    ICarbonLock carbonLock = CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier, lockType);
     LOGGER.info("Trying to acquire lock: " + carbonLock);
     if (carbonLock.lockWithRetries()) {
       LOGGER.info("Successfully acquired the lock " + carbonLock);
@@ -82,7 +86,7 @@ public class CarbonLockUtil {
   /**
    * Get and lock with default error message
    */
-  public static ICarbonLock getLockObject(CarbonTableIdentifier identifier, String lockType) {
+  public static ICarbonLock getLockObject(AbsoluteTableIdentifier identifier, String lockType) {
     return getLockObject(identifier,
         lockType,
         "Acquire table lock failed after retry, please try after some time");

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/locks/HdfsFileLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/HdfsFileLock.java b/core/src/main/java/org/apache/carbondata/core/locks/HdfsFileLock.java
index 752c70d..1a46770 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/HdfsFileLock.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/HdfsFileLock.java
@@ -25,10 +25,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 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.util.CarbonProperties;
-
-import org.apache.hadoop.conf.Configuration;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 
 /**
  * This class is used to handle the HDFS File locking.
@@ -47,26 +44,12 @@ public class HdfsFileLock extends AbstractCarbonLock {
 
   private static String tmpPath;
 
-  static {
-    Configuration conf = new Configuration(true);
-    String hdfsPath = conf.get(CarbonCommonConstants.FS_DEFAULT_FS);
-    // By default, we put the hdfs lock meta file for one table inside this table's store folder.
-    // If can not get the STORE_LOCATION, then use hadoop.tmp.dir .
-    tmpPath = CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION,
-               System.getProperty(CarbonCommonConstants.HDFS_TEMP_LOCATION));
-    if (!tmpPath.startsWith(CarbonCommonConstants.HDFSURL_PREFIX) && !tmpPath
-        .startsWith(CarbonCommonConstants.VIEWFSURL_PREFIX) && !tmpPath
-        .startsWith(CarbonCommonConstants.ALLUXIOURL_PREFIX)) {
-      tmpPath = hdfsPath + tmpPath;
-    }
-  }
-
   /**
    * @param lockFileLocation
    * @param lockFile
    */
   public HdfsFileLock(String lockFileLocation, String lockFile) {
-    this.location = tmpPath + CarbonCommonConstants.FILE_SEPARATOR + lockFileLocation
+    this.location = lockFileLocation
         + CarbonCommonConstants.FILE_SEPARATOR + lockFile;
     LOGGER.info("HDFS lock path:" + this.location);
     initRetry();
@@ -81,12 +64,11 @@ public class HdfsFileLock extends AbstractCarbonLock {
   }
 
   /**
-   * @param tableIdentifier
+   * @param absoluteTableIdentifier
    * @param lockFile
    */
-  public HdfsFileLock(CarbonTableIdentifier tableIdentifier, String lockFile) {
-    this(tableIdentifier.getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR + tableIdentifier
-        .getTableName(), lockFile);
+  public HdfsFileLock(AbsoluteTableIdentifier absoluteTableIdentifier, String lockFile) {
+    this(absoluteTableIdentifier.getTablePath(), lockFile);
   }
 
   /* (non-Javadoc)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java b/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
index d357a11..c3dfb57 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
@@ -28,8 +28,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 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.util.CarbonProperties;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 
 /**
  * This class handles the file locking in the local file system.
@@ -61,8 +60,6 @@ public class LocalFileLock extends AbstractCarbonLock {
    */
   private String lockFile;
 
-  public static final String tmpPath;
-
   private  String lockFilePath;
 
   /**
@@ -71,17 +68,14 @@ public class LocalFileLock extends AbstractCarbonLock {
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(LocalFileLock.class.getName());
 
-  static {
-    tmpPath = CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION,
-        System.getProperty("java.io.tmpdir"));
-  }
+
 
   /**
    * @param lockFileLocation
    * @param lockFile
    */
   public LocalFileLock(String lockFileLocation, String lockFile) {
-    this.location = tmpPath + CarbonCommonConstants.FILE_SEPARATOR + lockFileLocation;
+    this.location = lockFileLocation;
     this.lockFile = lockFile;
     initRetry();
   }
@@ -90,9 +84,8 @@ public class LocalFileLock extends AbstractCarbonLock {
    * @param tableIdentifier
    * @param lockFile
    */
-  public LocalFileLock(CarbonTableIdentifier tableIdentifier, String lockFile) {
-    this(tableIdentifier.getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR + tableIdentifier
-        .getTableName(), lockFile);
+  public LocalFileLock(AbsoluteTableIdentifier tableIdentifier, String lockFile) {
+    this(tableIdentifier.getTablePath(), lockFile);
     initRetry();
   }
 
@@ -103,8 +96,8 @@ public class LocalFileLock extends AbstractCarbonLock {
    */
   @Override public boolean lock() {
     try {
-      if (!FileFactory.isFileExist(location, FileFactory.getFileType(tmpPath))) {
-        FileFactory.mkdirs(location, FileFactory.getFileType(tmpPath));
+      if (!FileFactory.isFileExist(location, FileFactory.getFileType(location))) {
+        FileFactory.mkdirs(location, FileFactory.getFileType(location));
       }
       lockFilePath = location + CarbonCommonConstants.FILE_SEPARATOR +
           lockFile;
@@ -151,7 +144,7 @@ public class LocalFileLock extends AbstractCarbonLock {
           fileOutputStream.close();
           // deleting the lock file after releasing the lock.
           CarbonFile lockFile = FileFactory
-                  .getCarbonFile(lockFilePath, FileFactory.getFileType(lockFilePath));
+              .getCarbonFile(lockFilePath, FileFactory.getFileType(lockFilePath));
           if (!lockFile.exists() || lockFile.delete()) {
             LOGGER.info("Successfully deleted the lock file " + lockFilePath);
           } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/locks/ZooKeeperLocking.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/ZooKeeperLocking.java b/core/src/main/java/org/apache/carbondata/core/locks/ZooKeeperLocking.java
index 256c059..6fc2486 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/ZooKeeperLocking.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/ZooKeeperLocking.java
@@ -23,7 +23,7 @@ import java.util.List;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.util.CarbonProperties;
 
 import org.apache.zookeeper.CreateMode;
@@ -68,8 +68,9 @@ public class ZooKeeperLocking extends AbstractCarbonLock {
 
   private String lockTypeFolder;
 
-  public ZooKeeperLocking(CarbonTableIdentifier tableIdentifier, String lockFile) {
-    this(tableIdentifier.getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR + tableIdentifier
+  public ZooKeeperLocking(AbsoluteTableIdentifier absoluteTableIdentifier, String lockFile) {
+    this(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName()
+        + CarbonCommonConstants.FILE_SEPARATOR + absoluteTableIdentifier.getCarbonTableIdentifier()
         .getTableName(), lockFile);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
index 2203b3b..5862933 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
@@ -17,11 +17,11 @@
 
 package org.apache.carbondata.core.memory;
 
-import javax.annotation.concurrent.GuardedBy;
 import java.lang.ref.WeakReference;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Map;
+import javax.annotation.concurrent.GuardedBy;
 
 /**
  * Code ported from Apache Spark {org.apache.spark.unsafe.memory} package

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
index 1fbf544..d5434d8 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
@@ -32,10 +32,9 @@ public class AbsoluteTableIdentifier implements Serializable {
   private static final long serialVersionUID = 4695047103484427506L;
 
   /**
-   * path of the store
+   * path of the table
    */
-  private String storePath;
-
+  private String tablePath;
 
   private boolean isLocalPath;
 
@@ -45,62 +44,26 @@ public class AbsoluteTableIdentifier implements Serializable {
    */
   private CarbonTableIdentifier carbonTableIdentifier;
 
-  public AbsoluteTableIdentifier(String storePath, CarbonTableIdentifier carbonTableIdentifier) {
+  public AbsoluteTableIdentifier(String tablePath, CarbonTableIdentifier carbonTableIdentifier) {
     //TODO this should be moved to common place where path handling will be handled
-    this.storePath = FileFactory.getUpdatedFilePath(storePath);
-    isLocalPath = storePath.startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX);
+    this.tablePath = FileFactory.getUpdatedFilePath(tablePath);
+    isLocalPath = tablePath.startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX);
     this.carbonTableIdentifier = carbonTableIdentifier;
   }
 
   /**
-   * @return the storePath
-   */
-  public String getStorePath() {
-    return storePath;
-  }
-
-  /**
    * @return the carbonTableIdentifier
    */
   public CarbonTableIdentifier getCarbonTableIdentifier() {
     return carbonTableIdentifier;
   }
 
-  public static AbsoluteTableIdentifier from(String storePath, String dbName, String tableName) {
+  public static AbsoluteTableIdentifier from(String tablePath, String dbName, String tableName) {
     CarbonTableIdentifier identifier = new CarbonTableIdentifier(dbName, tableName, "");
-    return new AbsoluteTableIdentifier(storePath, identifier);
+    return new AbsoluteTableIdentifier(tablePath, identifier);
   }
-
-  /**
-   * By using the tablePath this method will prepare a AbsoluteTableIdentifier with
-   * dummy tableId(Long.toString(System.currentTimeMillis()).
-   * This instance could not be used to uniquely identify the table, this is just
-   * to get the database name, table name and store path to load the schema.
-   * @param tablePath
-   * @return returns AbsoluteTableIdentifier with dummy tableId
-   */
-  public static AbsoluteTableIdentifier fromTablePath(String tablePath) {
-    String formattedTablePath = tablePath.replace('\\', '/');
-    String[] names = formattedTablePath.split("/");
-    if (names.length < 3) {
-      throw new IllegalArgumentException("invalid table path: " + tablePath);
-    }
-
-    String tableName = names[names.length - 1];
-    String dbName = names[names.length - 2];
-    String storePath = formattedTablePath.substring(0,
-        formattedTablePath.lastIndexOf(dbName + CarbonCommonConstants.FILE_SEPARATOR + tableName)
-            - 1);
-
-    CarbonTableIdentifier identifier =
-        new CarbonTableIdentifier(dbName, tableName, Long.toString(System.currentTimeMillis()));
-    return new AbsoluteTableIdentifier(storePath, identifier);
-  }
-
   public String getTablePath() {
-    return getStorePath() + CarbonCommonConstants.FILE_SEPARATOR + getCarbonTableIdentifier()
-        .getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR + getCarbonTableIdentifier()
-        .getTableName();
+    return tablePath;
   }
 
   public String appendWithLocalPrefix(String path) {
@@ -119,7 +82,7 @@ public class AbsoluteTableIdentifier implements Serializable {
     int result = 1;
     result =
         prime * result + ((carbonTableIdentifier == null) ? 0 : carbonTableIdentifier.hashCode());
-    result = prime * result + ((storePath == null) ? 0 : storePath.hashCode());
+    result = prime * result + ((tablePath == null) ? 0 : tablePath.hashCode());
     return result;
   }
 
@@ -147,17 +110,17 @@ public class AbsoluteTableIdentifier implements Serializable {
     } else if (!carbonTableIdentifier.equals(other.carbonTableIdentifier)) {
       return false;
     }
-    if (storePath == null) {
-      if (other.storePath != null) {
+    if (tablePath == null) {
+      if (other.tablePath != null) {
         return false;
       }
-    } else if (!storePath.equals(other.storePath)) {
+    } else if (!tablePath.equals(other.tablePath)) {
       return false;
     }
     return true;
   }
 
   public String uniqueName() {
-    return storePath + "/" + carbonTableIdentifier.toString().toLowerCase();
+    return tablePath + "/" + carbonTableIdentifier.toString().toLowerCase();
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index 2027df2..0d60584 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -608,7 +608,7 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
    */
   @Override public TableInfo fromExternalToWrapperTableInfo(
       org.apache.carbondata.format.TableInfo externalTableInfo, String dbName, String tableName,
-      String storePath) {
+      String tablePath) {
     TableInfo wrapperTableInfo = new TableInfo();
     List<org.apache.carbondata.format.SchemaEvolutionEntry> schemaEvolutionList =
         externalTableInfo.getFact_table().getSchema_evolution().getSchema_evolution_history();
@@ -617,7 +617,7 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
             .getTime_stamp());
     wrapperTableInfo.setDatabaseName(dbName);
     wrapperTableInfo.setTableUniqueName(dbName + "_" + tableName);
-    wrapperTableInfo.setStorePath(storePath);
+    wrapperTableInfo.setTablePath(tablePath);
     wrapperTableInfo.setFactTable(
         fromExternalToWrapperTableSchema(externalTableInfo.getFact_table(), tableName));
     if (null != externalTableInfo.getDataMapSchemas()) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 0fd9fbf..f76ddc9 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -368,8 +368,8 @@ public class CarbonTable implements Serializable {
   /**
    * @return storepath
    */
-  public String getStorePath() {
-    return absoluteTableIdentifier.getStorePath();
+  public String getTablePath() {
+    return absoluteTableIdentifier.getTablePath();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
index 65878bc..40ce92d 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
@@ -76,7 +76,7 @@ public class TableInfo implements Serializable, Writable {
   /**
    * store location
    */
-  private String storePath;
+  private String tablePath;
 
   // this idenifier is a lazy field which will be created when it is used first time
   private AbsoluteTableIdentifier identifier;
@@ -176,12 +176,12 @@ public class TableInfo implements Serializable, Writable {
     this.metaDataFilepath = metaDataFilepath;
   }
 
-  public String getStorePath() {
-    return storePath;
+  public String getTablePath() {
+    return tablePath;
   }
 
-  public void setStorePath(String storePath) {
-    this.storePath = storePath;
+  public void setTablePath(String tablePath) {
+    this.tablePath = tablePath;
   }
 
   public List<DataMapSchema> getDataMapSchemaList() {
@@ -259,7 +259,7 @@ public class TableInfo implements Serializable, Writable {
     factTable.write(out);
     out.writeLong(lastUpdatedTime);
     out.writeUTF(metaDataFilepath);
-    out.writeUTF(storePath);
+    out.writeUTF(tablePath);
     boolean isChildSchemaExists =
         null != dataMapSchemaList && dataMapSchemaList.size() > 0;
     out.writeBoolean(isChildSchemaExists);
@@ -287,7 +287,7 @@ public class TableInfo implements Serializable, Writable {
     this.factTable.readFields(in);
     this.lastUpdatedTime = in.readLong();
     this.metaDataFilepath = in.readUTF();
-    this.storePath = in.readUTF();
+    this.tablePath = in.readUTF();
     boolean isChildSchemaExists = in.readBoolean();
     this.dataMapSchemaList = new ArrayList<>();
     if (isChildSchemaExists) {
@@ -319,7 +319,7 @@ public class TableInfo implements Serializable, Writable {
     if (identifier == null) {
       CarbonTableIdentifier carbontableIdentifier =
           new CarbonTableIdentifier(databaseName, factTable.getTableName(), factTable.getTableId());
-      identifier = new AbsoluteTableIdentifier(storePath, carbontableIdentifier);
+      identifier = new AbsoluteTableIdentifier(tablePath, carbontableIdentifier);
     }
     return identifier;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
index 6589ee5..29cf62a 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
@@ -195,7 +195,7 @@ public class CarbonUpdateUtil {
     AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier();
 
     CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+            .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
                     absoluteTableIdentifier.getCarbonTableIdentifier());
 
     String tableStatusPath = carbonTablePath.getTableStatusFilePath();
@@ -298,7 +298,7 @@ public class CarbonUpdateUtil {
     AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier();
 
     CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+            .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
                     absoluteTableIdentifier.getCarbonTableIdentifier());
     // as of now considering only partition 0.
     String partitionId = "0";
@@ -492,7 +492,7 @@ public class CarbonUpdateUtil {
     SegmentStatusManager ssm = new SegmentStatusManager(table.getAbsoluteTableIdentifier());
 
     CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(table.getAbsoluteTableIdentifier().getStorePath(),
+            .getCarbonTablePath(table.getAbsoluteTableIdentifier().getTablePath(),
                     table.getAbsoluteTableIdentifier().getCarbonTableIdentifier());
 
     LoadMetadataDetails[] details = ssm.readLoadMetadata(table.getMetaDataFilepath());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
index e09fda0..fa3c1eb 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
@@ -23,7 +23,6 @@ import java.util.List;
 
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-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.path.CarbonTablePath;
@@ -37,16 +36,6 @@ import org.apache.thrift.TBase;
 public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetadataReader {
 
   /**
-   * carbon table identifier
-   */
-  protected CarbonTableIdentifier carbonTableIdentifier;
-
-  /**
-   * carbon dictionary meta data store path
-   */
-  protected String storePath;
-
-  /**
    * column identifier
    */
   protected DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
@@ -64,15 +53,10 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
   /**
    * Constructor
    *
-   * @param storePath             carbon dictionary meta 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 CarbonDictionaryMetadataReaderImpl(String storePath,
-      CarbonTableIdentifier carbonTableIdentifier,
+  public CarbonDictionaryMetadataReaderImpl(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    this.storePath = storePath;
-    this.carbonTableIdentifier = carbonTableIdentifier;
     this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
     initFileLocation();
   }
@@ -171,7 +155,7 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
   protected void initFileLocation() {
     PathService pathService = CarbonCommonFactory.getPathService();
     CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(this.storePath, carbonTableIdentifier,
+        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
             dictionaryColumnUniqueIdentifier);
     this.columnDictionaryMetadataFilePath = carbonTablePath.getDictionaryMetaFilePath(
         dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
index 0325cf6..539ee16 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
@@ -25,7 +25,6 @@ import java.util.List;
 
 import org.apache.carbondata.core.cache.dictionary.ColumnDictionaryChunkIterator;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-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.path.CarbonTablePath;
@@ -40,16 +39,6 @@ import org.apache.thrift.TBase;
 public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
 
   /**
-   * carbon table identifier
-   */
-  protected CarbonTableIdentifier carbonTableIdentifier;
-
-  /**
-   * carbon dictionary data store path
-   */
-  protected String storePath;
-
-  /**
    * column name
    */
   protected DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
@@ -67,14 +56,10 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
   /**
    * 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
    */
-  public CarbonDictionaryReaderImpl(String storePath, CarbonTableIdentifier carbonTableIdentifier,
+  public CarbonDictionaryReaderImpl(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    this.storePath = storePath;
-    this.carbonTableIdentifier = carbonTableIdentifier;
     this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
     initFileLocation();
   }
@@ -217,7 +202,7 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
   protected void initFileLocation() {
     PathService pathService = CarbonCommonFactory.getPathService();
     CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(this.storePath, carbonTableIdentifier,
+        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
             dictionaryColumnUniqueIdentifier);
     this.columnDictionaryFilePath = carbonTablePath.getDictionaryFilePath(
         dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
@@ -303,8 +288,7 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
    * @return
    */
   protected CarbonDictionaryMetadataReader getDictionaryMetadataReader() {
-    return new CarbonDictionaryMetadataReaderImpl(this.storePath, carbonTableIdentifier,
-        this.dictionaryColumnUniqueIdentifier);
+    return new CarbonDictionaryMetadataReaderImpl(this.dictionaryColumnUniqueIdentifier);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
index 8c06434..12426f4 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
@@ -23,7 +23,6 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 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;
@@ -41,21 +40,11 @@ import org.apache.thrift.TBase;
 public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySortIndexReader {
 
   /**
-   * carbonTable Identifier holding the info of databaseName and tableName
-   */
-  protected CarbonTableIdentifier carbonTableIdentifier;
-
-  /**
    * column name
    */
   protected DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
 
   /**
-   * store location
-   */
-  protected String carbonStorePath;
-
-  /**
    * the path of the dictionary Sort Index file
    */
   protected String sortIndexFilePath;
@@ -77,17 +66,11 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
   private ThriftReader dictionarySortIndexThriftReader;
 
   /**
-   * @param carbonTableIdentifier            Carbon Table identifier holding the database name
-   *                                         and table name
    * @param dictionaryColumnUniqueIdentifier column name
-   * @param carbonStorePath                  carbon store path
    */
-  public CarbonDictionarySortIndexReaderImpl(final CarbonTableIdentifier carbonTableIdentifier,
-      final DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier,
-      final String carbonStorePath) {
-    this.carbonTableIdentifier = carbonTableIdentifier;
+  public CarbonDictionarySortIndexReaderImpl(
+      final DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
     this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
-    this.carbonStorePath = carbonStorePath;
   }
 
   /**
@@ -154,7 +137,7 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
   protected void initPath() {
     PathService pathService = CarbonCommonFactory.getPathService();
     CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(carbonStorePath, carbonTableIdentifier,
+        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
             dictionaryColumnUniqueIdentifier);
     try {
       CarbonDictionaryColumnMetaChunk chunkMetaObjectForLastSegmentEntry =
@@ -196,8 +179,7 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
    * @return
    */
   protected CarbonDictionaryMetadataReader getDictionaryMetadataReader() {
-    return new CarbonDictionaryMetadataReaderImpl(carbonStorePath, carbonTableIdentifier,
-        dictionaryColumnUniqueIdentifier);
+    return new CarbonDictionaryMetadataReaderImpl(dictionaryColumnUniqueIdentifier);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
index 251a06b..88a8238 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
@@ -145,8 +145,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
       // get the table blocks
       CacheProvider cacheProvider = CacheProvider.getInstance();
       BlockIndexStore<TableBlockUniqueIdentifier, AbstractIndex> cache =
-          (BlockIndexStore) cacheProvider
-              .createCache(CacheType.EXECUTOR_BTREE, queryModel.getTable().getStorePath());
+          (BlockIndexStore) cacheProvider.createCache(CacheType.EXECUTOR_BTREE);
       // remove the invalid table blocks, block which is deleted or compacted
       cache.removeTableBlocks(queryModel.getInvalidSegmentIds(),
           queryModel.getAbsoluteTableIdentifier());
@@ -260,7 +259,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
             queryModel.getQueryDimension(), tableBlockDimensions,
             segmentProperties.getComplexDimensions());
     int tableFactPathLength = CarbonStorePath
-        .getCarbonTablePath(queryModel.getAbsoluteTableIdentifier().getStorePath(),
+        .getCarbonTablePath(queryModel.getAbsoluteTableIdentifier().getTablePath(),
             queryModel.getAbsoluteTableIdentifier().getCarbonTableIdentifier()).getFactDir()
         .length() + 1;
     blockExecutionInfo.setBlockId(filePath.substring(tableFactPathLength));


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

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java b/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
index 8bb2052..86ba9c8 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
@@ -33,6 +33,7 @@ import java.util.Properties;
 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.util.path.CarbonStorePath;
@@ -65,6 +66,8 @@ public class CarbonDictionaryWriterImplTest {
 
   private CarbonTableIdentifier carbonTableIdentifier;
 
+  private AbsoluteTableIdentifier absoluteTableIdentifier;
+
   private String databaseName;
 
   private String tableName;
@@ -100,8 +103,9 @@ public class CarbonDictionaryWriterImplTest {
     this.carbonStorePath = props.getProperty("storePath", "carbonStore");
     this.columnIdentifier = new ColumnIdentifier("Name", null, null);
     carbonTableIdentifier = new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
+    absoluteTableIdentifier = new AbsoluteTableIdentifier(carbonStorePath, carbonTableIdentifier);
     this.dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
             columnIdentifier.getDataType(),
             CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier));
     deleteStorePath();
@@ -183,8 +187,7 @@ public class CarbonDictionaryWriterImplTest {
    */
   private CarbonDictionaryWriterImpl prepareWriter() throws IOException {
     initDictionaryDirPaths();
-    return new CarbonDictionaryWriterImpl(this.carbonStorePath, carbonTableIdentifier,
-        dictionaryColumnUniqueIdentifier);
+    return new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
   }
 
   /**
@@ -438,8 +441,7 @@ public class CarbonDictionaryWriterImplTest {
    */
   private List<CarbonDictionaryColumnMetaChunk> readDictionaryMetadataFile() throws IOException {
     CarbonDictionaryMetadataReaderImpl columnMetadataReaderImpl =
-        new CarbonDictionaryMetadataReaderImpl(this.carbonStorePath, this.carbonTableIdentifier,
-            this.dictionaryColumnUniqueIdentifier);
+        new CarbonDictionaryMetadataReaderImpl(this.dictionaryColumnUniqueIdentifier);
     List<CarbonDictionaryColumnMetaChunk> dictionaryMetaChunkList = null;
     // read metadata file
     try {
@@ -457,8 +459,7 @@ public class CarbonDictionaryWriterImplTest {
   private List<byte[]> readDictionaryFile(long dictionaryStartOffset, long dictionaryEndOffset)
       throws IOException {
     CarbonDictionaryReaderImpl dictionaryReader =
-        new CarbonDictionaryReaderImpl(this.carbonStorePath, this.carbonTableIdentifier,
-            this.dictionaryColumnUniqueIdentifier);
+        new CarbonDictionaryReaderImpl(this.dictionaryColumnUniqueIdentifier);
     List<byte[]> dictionaryValues = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     try {
       if (0 == dictionaryEndOffset) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
index d04d8a2..e64726a 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.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.reader.sortindex.CarbonDictionarySortIndexReader;
@@ -45,6 +46,7 @@ public class CarbonDictionarySortIndexWriterImplTest {
 
   private String storePath;
   private CarbonTableIdentifier carbonTableIdentifier = null;
+  private AbsoluteTableIdentifier absoluteTableIdentifier = null;
   private ColumnIdentifier columnIdentifier = null;
   private CarbonDictionaryWriter dictionaryWriter = null;
   private CarbonDictionarySortIndexWriter dictionarySortIndexWriter = null;
@@ -54,15 +56,21 @@ public class CarbonDictionarySortIndexWriterImplTest {
     storePath = "target/carbonStore";
     carbonTableIdentifier =
         new CarbonTableIdentifier("testSchema", "carbon", UUID.randomUUID().toString());
+    String tablePath =
+        storePath + "/" + carbonTableIdentifier.getDatabaseName() + "/" + carbonTableIdentifier
+            .getTableName();
+    absoluteTableIdentifier = new AbsoluteTableIdentifier(tablePath, carbonTableIdentifier);
     columnIdentifier = new ColumnIdentifier("Name", null, null);
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier = new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier, columnIdentifier.getDataType(),
-        CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier));
+    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
+            columnIdentifier.getDataType(),
+            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
     dictionaryWriter =
-        new CarbonDictionaryWriterImpl(storePath, carbonTableIdentifier, dictionaryColumnUniqueIdentifier);
+        new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
     dictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, dictionaryColumnUniqueIdentifier, storePath);
+        new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
     carbonDictionarySortIndexReader =
-        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, dictionaryColumnUniqueIdentifier, storePath);
+        new CarbonDictionarySortIndexReaderImpl(dictionaryColumnUniqueIdentifier);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheClient.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheClient.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheClient.java
index 4c5b359..8be1e2e 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheClient.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheClient.java
@@ -31,9 +31,9 @@ public class CacheClient {
   private CacheAccessClient<TableSegmentUniqueIdentifier, SegmentTaskIndexWrapper>
       segmentAccessClient;
 
-  public CacheClient(String storePath) {
+  public CacheClient() {
     Cache<TableSegmentUniqueIdentifier, SegmentTaskIndexWrapper> segmentCache =
-        CacheProvider.getInstance().createCache(CacheType.DRIVER_BTREE, storePath);
+        CacheProvider.getInstance().createCache(CacheType.DRIVER_BTREE);
     segmentAccessClient = new CacheAccessClient<>(segmentCache);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
index e5aac84..0aa2974 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
@@ -82,7 +82,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
@@ -96,7 +95,6 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.mapreduce.task.JobContextImpl;
-import org.apache.hadoop.util.StringUtils;
 
 /**
  * Carbon Input format class representing one carbon table
@@ -116,7 +114,8 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
   private static final String TABLE_INFO = "mapreduce.input.carboninputformat.tableinfo";
   private static final String CARBON_READ_SUPPORT = "mapreduce.input.carboninputformat.readsupport";
   private static final String CARBON_CONVERTER = "mapreduce.input.carboninputformat.converter";
-
+  private static final String DATABASE_NAME = "mapreduce.input.carboninputformat.databaseName";
+  private static final String TABLE_NAME = "mapreduce.input.carboninputformat.tableName";
   // a cache for carbon table, it will be used in task side
   private CarbonTable carbonTable;
 
@@ -313,11 +312,8 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
   private static AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configuration)
       throws IOException {
     String dirs = configuration.get(INPUT_DIR, "");
-    String[] inputPaths = StringUtils.split(dirs);
-    if (inputPaths.length == 0) {
-      throw new InvalidPathException("No input paths specified in job");
-    }
-    return AbsoluteTableIdentifier.fromTablePath(inputPaths[0]);
+    return AbsoluteTableIdentifier
+        .from(dirs, getDatabaseName(configuration), getTableName(configuration));
   }
 
   /**
@@ -331,7 +327,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
    */
   @Override public List<InputSplit> getSplits(JobContext job) throws IOException {
     AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration());
-    CacheClient cacheClient = new CacheClient(identifier.getStorePath());
+    CacheClient cacheClient = new CacheClient();
     try {
       List<String> invalidSegments = new ArrayList<>();
       List<UpdateVO> invalidTimestampsList = new ArrayList<>();
@@ -727,7 +723,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
    */
   public BlockMappingVO getBlockRowCount(JobContext job,
       AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException, KeyGenException {
-    CacheClient cacheClient = new CacheClient(absoluteTableIdentifier.getStorePath());
+    CacheClient cacheClient = new CacheClient();
     try {
       SegmentUpdateStatusManager updateStatusManager =
           new SegmentUpdateStatusManager(absoluteTableIdentifier);
@@ -978,4 +974,24 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
     return new String[] { "0" };
   }
 
+  public static void setDatabaseName(Configuration configuration, String databaseName) {
+    if (null != databaseName) {
+      configuration.set(DATABASE_NAME, databaseName);
+    }
+  }
+
+  public static String getDatabaseName(Configuration configuration) {
+    return configuration.get(DATABASE_NAME);
+  }
+
+  public static void setTableName(Configuration configuration, String tableName) {
+    if (null != tableName) {
+      configuration.set(TABLE_NAME, tableName);
+    }
+  }
+
+  public static String getTableName(Configuration configuration) {
+    return configuration.get(TABLE_NAME);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index f3963ad..6e840e2 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -38,6 +38,7 @@ import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.TableDataMap;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.exception.InvalidConfigurationException;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
@@ -88,7 +89,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -98,7 +98,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.hadoop.mapreduce.security.TokenCache;
-import org.apache.hadoop.util.StringUtils;
 
 /**
  * Input format of CarbonData file.
@@ -123,6 +122,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   private static final String CARBON_READ_SUPPORT = "mapreduce.input.carboninputformat.readsupport";
   private static final String CARBON_CONVERTER = "mapreduce.input.carboninputformat.converter";
   private static final String DATA_MAP_DSTR = "mapreduce.input.carboninputformat.datamapdstr";
+  public static final String DATABASE_NAME = "mapreduce.input.carboninputformat.databaseName";
+  public static final String TABLE_NAME = "mapreduce.input.carboninputformat.tableName";
 
   // a cache for carbon table, it will be used in task side
   private CarbonTable carbonTable;
@@ -288,12 +289,13 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
 
   private AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configuration)
       throws IOException {
-    String dirs = configuration.get(INPUT_DIR, "");
-    String[] inputPaths = StringUtils.split(dirs);
-    if (inputPaths.length == 0) {
-      throw new InvalidPathException("No input paths specified in job");
+    String tablePath = configuration.get(INPUT_DIR, "");
+    try {
+      return AbsoluteTableIdentifier
+          .from(tablePath, getDatabaseName(configuration), getTableName(configuration));
+    } catch (InvalidConfigurationException e) {
+      throw new IOException(e);
     }
-    return AbsoluteTableIdentifier.fromTablePath(inputPaths[0]);
   }
 
   /**
@@ -941,4 +943,34 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     }
     return (DataTypeConverter) ObjectSerializationUtil.convertStringToObject(converter);
   }
+
+  public static void setDatabaseName(Configuration configuration, String databaseName) {
+    if (null != databaseName) {
+      configuration.set(DATABASE_NAME, databaseName);
+    }
+  }
+
+  public static String getDatabaseName(Configuration configuration)
+      throws InvalidConfigurationException {
+    String databseName = configuration.get(DATABASE_NAME);
+    if (null == databseName) {
+      throw new InvalidConfigurationException("Database name is not set.");
+    }
+    return databseName;
+  }
+
+  public static void setTableName(Configuration configuration, String tableName) {
+    if (null != tableName) {
+      configuration.set(TABLE_NAME, tableName);
+    }
+  }
+
+  public static String getTableName(Configuration configuration)
+      throws InvalidConfigurationException {
+    String tableName = configuration.get(TABLE_NAME);
+    if (tableName == null) {
+      throw new InvalidConfigurationException("Table name is not set");
+    }
+    return tableName;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
index 66a06ba..1b875bc 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
@@ -99,8 +99,6 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl
       public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
           throws IOException, InterruptedException {
         DataMapDistributable distributable = (DataMapDistributable)inputSplit;
-        AbsoluteTableIdentifier identifier =
-            AbsoluteTableIdentifier.fromTablePath(distributable.getTablePath());
         TableDataMap dataMap = DataMapStoreManager.getInstance()
             .getDataMap(identifier, distributable.getDataMapName(),
                 distributable.getDataMapFactoryClass());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
index 4268ee2..37796db 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
@@ -98,7 +98,7 @@ class InMemoryBTreeIndex implements Index {
   private Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> getSegmentAbstractIndexs(
       JobContext job, AbsoluteTableIdentifier identifier) throws IOException {
     Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> segmentIndexMap = null;
-    CacheClient cacheClient = new CacheClient(identifier.getStorePath());
+    CacheClient cacheClient = new CacheClient();
     TableSegmentUniqueIdentifier segmentUniqueIdentifier =
         new TableSegmentUniqueIdentifier(identifier, segment.getId());
     try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
index 32d879f..f4927dd 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
@@ -61,10 +61,10 @@ public class DictionaryDecodeReadSupport<T> implements CarbonReadSupport<T> {
           .hasEncoding(Encoding.DIRECT_DICTIONARY) && !carbonColumns[i].isComplex()) {
         CacheProvider cacheProvider = CacheProvider.getInstance();
         Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache = cacheProvider
-            .createCache(CacheType.FORWARD_DICTIONARY, absoluteTableIdentifier.getStorePath());
+            .createCache(CacheType.FORWARD_DICTIONARY);
         dataTypes[i] = carbonColumns[i].getDataType();
         dictionaries[i] = forwardDictionaryCache.get(new DictionaryColumnUniqueIdentifier(
-            absoluteTableIdentifier.getCarbonTableIdentifier(),
+            absoluteTableIdentifier,
             carbonColumns[i].getColumnIdentifier(), dataTypes[i],
             CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)));
       } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
index eb07f7e..e6feb93 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
@@ -96,7 +96,7 @@ public class CarbonStreamInputFormat extends FileInputFormat<Void, Object> {
         boolean isDirectDictionary =
             CarbonUtil.hasEncoding(child.getEncoder(), Encoding.DIRECT_DICTIONARY);
         DictionaryColumnUniqueIdentifier dictionarIdentifier =
-            new DictionaryColumnUniqueIdentifier(carbontable.getCarbonTableIdentifier(),
+            new DictionaryColumnUniqueIdentifier(carbontable.getAbsoluteTableIdentifier(),
                 child.getColumnIdentifier(), child.getDataType(),
                 CarbonStorePath.getCarbonTablePath(carbontable.getAbsoluteTableIdentifier()));
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
index 4e3e6cf..a22461d 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
@@ -279,7 +279,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
         fileSplit.getStart() == 0);
 
     cacheProvider = CacheProvider.getInstance();
-    cache = cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, carbonTable.getStorePath());
+    cache = cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);
     queryTypes = CarbonStreamInputFormat.getComplexDimensions(carbonTable, storageColumns, cache);
 
     outputSchema = new StructType(CarbonTypeUtil.convertCarbonSchemaToSparkSchema(projection));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
index b4444be..630828a 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -82,6 +82,10 @@ public class CarbonInputFormatUtil {
       AbsoluteTableIdentifier identifier,
       Job job) throws IOException {
     CarbonTableInputFormat<V> carbonInputFormat = new CarbonTableInputFormat<>();
+    carbonInputFormat.setDatabaseName(job.getConfiguration(),
+        identifier.getCarbonTableIdentifier().getDatabaseName());
+    carbonInputFormat
+        .setTableName(job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName());
     FileInputFormat.addInputPath(job, new Path(identifier.getTablePath()));
     return carbonInputFormat;
   }
@@ -90,6 +94,10 @@ public class CarbonInputFormatUtil {
       AbsoluteTableIdentifier identifier, List<String> partitionId, Job job) throws IOException {
     CarbonTableInputFormat<V> carbonTableInputFormat = new CarbonTableInputFormat<>();
     carbonTableInputFormat.setPartitionIdList(job.getConfiguration(), partitionId);
+    carbonTableInputFormat.setDatabaseName(job.getConfiguration(),
+        identifier.getCarbonTableIdentifier().getDatabaseName());
+    carbonTableInputFormat
+        .setTableName(job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName());
     FileInputFormat.addInputPath(job, new Path(identifier.getTablePath()));
     return carbonTableInputFormat;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
index f6c9e59..f1ce324 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
@@ -61,7 +61,7 @@ public class SchemaReader {
       TableInfo wrapperTableInfo = schemaConverter
           .fromExternalToWrapperTableInfo(tableInfo,
               identifier.getCarbonTableIdentifier().getDatabaseName(), tableName,
-              identifier.getStorePath());
+              identifier.getTablePath());
       wrapperTableInfo.setMetaDataFilepath(CarbonTablePath.getFolderContainingFile(schemaFilePath));
       CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo);
       return CarbonMetadata.getInstance().getCarbonTable(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
index 865dabe..068d8b3 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
@@ -189,6 +189,10 @@ public class CarbonInputMapperTest extends TestCase {
     if (filter != null) {
       CarbonInputFormat.setFilterPredicates(job.getConfiguration(), filter);
     }
+    CarbonInputFormat.setDatabaseName(job.getConfiguration(),
+        abs.getCarbonTableIdentifier().getDatabaseName());
+    CarbonInputFormat.setTableName(job.getConfiguration(),
+        abs.getCarbonTableIdentifier().getTableName());
     FileInputFormat.addInputPath(job, new Path(abs.getTablePath()));
     CarbonUtil.deleteFoldersAndFiles(new File(outPath + "1"));
     FileOutputFormat.setOutputPath(job, new Path(outPath + "1"));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
index 3b5d736..b4145ef 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
@@ -99,14 +99,15 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 public class StoreCreator {
 
   private static AbsoluteTableIdentifier absoluteTableIdentifier;
+  private static String storePath = null;
 
   static {
     try {
-      String storePath = new File("target/store").getCanonicalPath();
+      storePath = new File("target/store").getCanonicalPath();
       String dbName = "testdb";
       String tableName = "testtable";
       absoluteTableIdentifier =
-          new AbsoluteTableIdentifier(storePath, new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
+          new AbsoluteTableIdentifier(storePath +"/testdb/testtable", new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
     } catch (IOException ex) {
 
     }
@@ -126,7 +127,7 @@ public class StoreCreator {
     loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
     loadModel.setFactFilePath(factFilePath);
     loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
-    loadModel.setStorePath(absoluteTableIdentifier.getStorePath());
+    loadModel.setTablePath(absoluteTableIdentifier.getTablePath());
     loadModel.setDateFormat(null);
     loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
         CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
@@ -163,17 +164,17 @@ public class StoreCreator {
     try {
       String factFilePath =
           new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
-      File storeDir = new File(absoluteTableIdentifier.getStorePath());
+      File storeDir = new File(storePath);
       CarbonUtil.deleteFoldersAndFiles(storeDir);
       CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS,
-          absoluteTableIdentifier.getStorePath());
+          storePath);
 
       CarbonTable table = createTable(absoluteTableIdentifier);
       writeDictionary(factFilePath, table);
       CarbonLoadModel loadModel =
           buildCarbonLoadModel(table, factFilePath, absoluteTableIdentifier);
 
-      executeGraph(loadModel, absoluteTableIdentifier.getStorePath());
+      executeGraph(loadModel, storePath);
 
     } catch (Exception e) {
       e.printStackTrace();
@@ -183,7 +184,7 @@ public class StoreCreator {
   public static CarbonTable createTable(
       AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException {
     TableInfo tableInfo = new TableInfo();
-    tableInfo.setStorePath(absoluteTableIdentifier.getStorePath());
+    tableInfo.setTablePath(absoluteTableIdentifier.getTablePath());
     tableInfo.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
     TableSchema tableSchema = new TableSchema();
     tableSchema.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
@@ -276,8 +277,7 @@ public class StoreCreator {
     tableInfo.setLastUpdatedTime(System.currentTimeMillis());
     tableInfo.setFactTable(tableSchema);
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
+        .getCarbonTablePath(absoluteTableIdentifier);
     String schemaFilePath = carbonTablePath.getSchemaFilePath();
     String schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath);
     tableInfo.setMetaDataFilepath(schemaMetadataPath);
@@ -327,34 +327,33 @@ public class StoreCreator {
     }
 
     Cache dictCache = CacheProvider.getInstance()
-        .createCache(CacheType.REVERSE_DICTIONARY, absoluteTableIdentifier.getStorePath());
+        .createCache(CacheType.REVERSE_DICTIONARY);
     for (int i = 0; i < set.length; i++) {
       ColumnIdentifier columnIdentifier = new ColumnIdentifier(dims.get(i).getColumnId(), null, null);
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-          new DictionaryColumnUniqueIdentifier(table.getCarbonTableIdentifier(), columnIdentifier,
+          new DictionaryColumnUniqueIdentifier(table.getAbsoluteTableIdentifier(), columnIdentifier,
               columnIdentifier.getDataType(), CarbonStorePath
-              .getCarbonTablePath(table.getStorePath(), table.getCarbonTableIdentifier()));
+              .getCarbonTablePath(table.getAbsoluteTableIdentifier().getTablePath(),
+                  table.getCarbonTableIdentifier()));
       CarbonDictionaryWriter writer =
-          new CarbonDictionaryWriterImpl(absoluteTableIdentifier.getStorePath(),
-              absoluteTableIdentifier.getCarbonTableIdentifier(), dictionaryColumnUniqueIdentifier);
+          new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
       for (String value : set[i]) {
         writer.write(value);
       }
       writer.close();
       writer.commit();
       Dictionary dict = (Dictionary) dictCache.get(
-          new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier.getCarbonTableIdentifier(),
+          new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
         		  columnIdentifier, dims.get(i).getDataType(),CarbonStorePath
-              .getCarbonTablePath(table.getStorePath(), table.getCarbonTableIdentifier())));
+              .getCarbonTablePath(table.getAbsoluteTableIdentifier().getTablePath(),
+                  table.getCarbonTableIdentifier())));
       CarbonDictionarySortInfoPreparator preparator =
           new CarbonDictionarySortInfoPreparator();
       List<String> newDistinctValues = new ArrayList<String>();
       CarbonDictionarySortInfo dictionarySortInfo =
           preparator.getDictionarySortInfo(newDistinctValues, dict, dims.get(i).getDataType());
       CarbonDictionarySortIndexWriter carbonDictionaryWriter =
-          new CarbonDictionarySortIndexWriterImpl(
-              absoluteTableIdentifier.getCarbonTableIdentifier(), dictionaryColumnUniqueIdentifier,
-              absoluteTableIdentifier.getStorePath());
+          new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
       try {
         carbonDictionaryWriter.writeSortIndex(dictionarySortInfo.getSortIndex());
         carbonDictionaryWriter.writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted());
@@ -379,7 +378,7 @@ public class StoreCreator {
     String databaseName = loadModel.getDatabaseName();
     String tableName = loadModel.getTableName();
     String tempLocationKey = databaseName + '_' + tableName + "_1";
-    CarbonProperties.getInstance().addProperty(tempLocationKey, storeLocation);
+    CarbonProperties.getInstance().addProperty(tempLocationKey, storeLocation + "/" + databaseName + "/" + tableName);
     CarbonProperties.getInstance().addProperty("store_output_location", outPutLoc);
     CarbonProperties.getInstance().addProperty("send.signal.load", "false");
     CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true");
@@ -421,7 +420,7 @@ public class StoreCreator {
 
     CSVRecordReaderIterator readerIterator = new CSVRecordReaderIterator(recordReader, blockDetails, hadoopAttemptContext);
     new DataLoadExecutor().execute(loadModel,
-        new String[] {storeLocation},
+        new String[] {storeLocation + "/" + databaseName + "/" + tableName},
         new CarbonIterator[]{readerIterator});
 
     writeLoadMetadata(loadModel.getCarbonDataLoadSchema(), loadModel.getTableName(), loadModel.getTableName(),

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
----------------------------------------------------------------------
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
index 2a19271..2f770cd 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
@@ -35,7 +35,6 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
@@ -84,12 +83,11 @@ public class CarbonDictionaryDecodeReadSupport<T> implements CarbonReadSupport<T
           .hasEncoding(Encoding.DIRECT_DICTIONARY) && !carbonColumns[i].isComplex()) {
         CacheProvider cacheProvider = CacheProvider.getInstance();
         Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache = cacheProvider
-            .createCache(CacheType.FORWARD_DICTIONARY, absoluteTableIdentifier.getStorePath());
+            .createCache(CacheType.FORWARD_DICTIONARY);
         dataTypes[i] = carbonColumns[i].getDataType();
         dictionaries[i] = forwardDictionaryCache.get(
-            new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier.getCarbonTableIdentifier(),
-                carbonColumns[i].getColumnIdentifier(), dataTypes[i],
-                CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)));
+            new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
+                carbonColumns[i].getColumnIdentifier()));
       } else {
         dataTypes[i] = carbonColumns[i].getDataType();
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
index 4cbc692..aabd3df 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
@@ -77,8 +77,8 @@ public class MapredCarbonInputFormat extends CarbonInputFormat<ArrayWritable>
         }
       }
     }
-    AbsoluteTableIdentifier absoluteTableIdentifier =
-        AbsoluteTableIdentifier.fromTablePath(validInputPath);
+    AbsoluteTableIdentifier absoluteTableIdentifier = AbsoluteTableIdentifier
+        .from(validInputPath, getDatabaseName(configuration), getTableName(configuration));
     // read the schema file to get the absoluteTableIdentifier having the correct table id
     // persisted in the schema
     CarbonTable carbonTable = SchemaReader.readCarbonTableFromStore(absoluteTableIdentifier);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/hive/src/main/scala/org/apache/carbondata/hiveexample/HiveExample.scala
----------------------------------------------------------------------
diff --git a/integration/hive/src/main/scala/org/apache/carbondata/hiveexample/HiveExample.scala b/integration/hive/src/main/scala/org/apache/carbondata/hiveexample/HiveExample.scala
index 9c1d51e..11839c9 100644
--- a/integration/hive/src/main/scala/org/apache/carbondata/hiveexample/HiveExample.scala
+++ b/integration/hive/src/main/scala/org/apache/carbondata/hiveexample/HiveExample.scala
@@ -97,7 +97,7 @@ object HiveExample {
     statement
       .execute(
         "ALTER TABLE HIVE_CARBON_EXAMPLE SET LOCATION " +
-        s"'file:///$store/default/hive_carbon_example' ")
+        s"'file:///$store/hive_carbon_example' ")
 
     val sql = "SELECT * FROM HIVE_CARBON_EXAMPLE"
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
index e49dcee..65c7373 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
@@ -93,8 +93,7 @@ public class CarbondataRecordSetProvider implements ConnectorRecordSetProvider {
       Configuration conf = new Configuration();
       conf.set(CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS, "");
       String carbonTablePath = PathFactory.getInstance()
-          .getCarbonTablePath(targetTable.getAbsoluteTableIdentifier().getStorePath(),
-              targetTable.getCarbonTableIdentifier(), null).getPath();
+          .getCarbonTablePath(targetTable.getAbsoluteTableIdentifier(), null).getPath();
 
       conf.set(CarbonTableInputFormat.INPUT_DIR, carbonTablePath);
       JobConf jobConf = new JobConf(conf);
@@ -123,6 +122,10 @@ public class CarbondataRecordSetProvider implements ConnectorRecordSetProvider {
     try {
       CarbonTableInputFormat
           .setTablePath(conf, identifier.appendWithLocalPrefix(identifier.getTablePath()));
+      CarbonTableInputFormat
+          .setDatabaseName(conf, identifier.getCarbonTableIdentifier().getDatabaseName());
+      CarbonTableInputFormat
+          .setTableName(conf, identifier.getCarbonTableIdentifier().getTableName());
     } catch (IOException e) {
       throw new RuntimeException("Unable to create the CarbonTableInputFormat", e);
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
index 9839fc8..8e6abd4 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
@@ -293,14 +293,20 @@ public class CarbonTableReader {
       // If table is not previously cached, then:
 
       // Step 1: get store path of the table and cache it.
-      String storePath = config.getStorePath();
       // create table identifier. the table id is randomly generated.
       cache.carbonTableIdentifier =
               new CarbonTableIdentifier(table.getSchemaName(), table.getTableName(),
                       UUID.randomUUID().toString());
+      String storePath = config.getStorePath();
+      String tablePath = storePath + "/" + cache.carbonTableIdentifier.getDatabaseName() + "/"
+          + cache.carbonTableIdentifier.getTableName();
+
       // get the store path of the table.
+
+      AbsoluteTableIdentifier absoluteTableIdentifier =
+          new AbsoluteTableIdentifier(tablePath, cache.carbonTableIdentifier);
       cache.carbonTablePath =
-              PathFactory.getInstance().getCarbonTablePath(storePath, cache.carbonTableIdentifier, null);
+          PathFactory.getInstance().getCarbonTablePath(absoluteTableIdentifier, null);
       // cache the table
       cc.put(table, cache);
 
@@ -325,8 +331,8 @@ public class CarbonTableReader {
       SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
       // wrapperTableInfo is the code level information of a table in carbondata core, different from the Thrift TableInfo.
       TableInfo wrapperTableInfo = schemaConverter
-              .fromExternalToWrapperTableInfo(tableInfo, table.getSchemaName(), table.getTableName(),
-                      storePath);
+          .fromExternalToWrapperTableInfo(tableInfo, table.getSchemaName(), table.getTableName(),
+              tablePath);
       wrapperTableInfo.setMetaDataFilepath(
               CarbonTablePath.getFolderContainingFile(cache.carbonTablePath.getSchemaFilePath()));
 
@@ -354,9 +360,10 @@ public class CarbonTableReader {
     Configuration config = new Configuration();
     config.set(CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS, "");
     String carbonTablePath = PathFactory.getInstance()
-            .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier().getStorePath(),
-                    carbonTable.getCarbonTableIdentifier(), null).getPath();
+        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier(), null).getPath();
     config.set(CarbonTableInputFormat.INPUT_DIR, carbonTablePath);
+    config.set(CarbonTableInputFormat.DATABASE_NAME, carbonTable.getDatabaseName());
+    config.set(CarbonTableInputFormat.TABLE_NAME, carbonTable.getFactTableName());
 
     try {
       CarbonTableInputFormat.setTableInfo(config, tableInfo);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala b/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
index b9a9f0d..c8e74a3 100644
--- a/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
+++ b/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
@@ -59,11 +59,11 @@ class CarbonDictionaryDecodeReadSupport[T] extends CarbonReadSupport[T] {
         val cacheProvider: CacheProvider = CacheProvider.getInstance
         val forwardDictionaryCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
           cacheProvider
-            .createCache(CacheType.FORWARD_DICTIONARY, absoluteTableIdentifier.getStorePath)
+            .createCache(CacheType.FORWARD_DICTIONARY)
         dataTypes(index) = carbonColumn.getDataType
         dictionaries(index) = forwardDictionaryCache
-          .get(new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier
-            .getCarbonTableIdentifier, carbonColumn.getColumnIdentifier))
+          .get(new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
+            carbonColumn.getColumnIdentifier))
         dictionarySliceArray(index) = createSliceArrayBlock(dictionaries(index))
 
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index e8f5d6d..17a4188 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -71,16 +71,16 @@ object CarbonDataStoreCreator {
       val dbName: String = "testdb"
       val tableName: String = "testtable"
       val absoluteTableIdentifier = new AbsoluteTableIdentifier(
-        storePath,
+        storePath + "/"+ dbName + "/" + tableName,
         new CarbonTableIdentifier(dbName,
           tableName,
           UUID.randomUUID().toString))
       val factFilePath: String = new File(dataFilePath).getCanonicalPath
-      val storeDir: File = new File(absoluteTableIdentifier.getStorePath)
+      val storeDir: File = new File(absoluteTableIdentifier.getTablePath)
       CarbonUtil.deleteFoldersAndFiles(storeDir)
       CarbonProperties.getInstance.addProperty(
         CarbonCommonConstants.STORE_LOCATION_HDFS,
-        absoluteTableIdentifier.getStorePath)
+        absoluteTableIdentifier.getTablePath)
       val table: CarbonTable = createTable(absoluteTableIdentifier)
       writeDictionary(factFilePath, table, absoluteTableIdentifier)
       val schema: CarbonDataLoadSchema = new CarbonDataLoadSchema(table)
@@ -95,7 +95,7 @@ object CarbonDataStoreCreator {
         absoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
       loadModel.setFactFilePath(factFilePath)
       loadModel.setLoadMetadataDetails(new ArrayList[LoadMetadataDetails]())
-      loadModel.setStorePath(absoluteTableIdentifier.getStorePath)
+      loadModel.setTablePath(absoluteTableIdentifier.getTablePath)
       CarbonProperties.getInstance
         .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_COLUMN_PAGE_LOADING, "true")
 
@@ -131,7 +131,7 @@ object CarbonDataStoreCreator {
       loadModel.setPartitionId("0")
       loadModel.setFactTimeStamp(System.currentTimeMillis())
       loadModel.setMaxColumns("15")
-      executeGraph(loadModel, absoluteTableIdentifier.getStorePath)
+      executeGraph(loadModel, storePath)
     } catch {
       case e: Exception => e.printStackTrace()
 
@@ -140,7 +140,7 @@ object CarbonDataStoreCreator {
 
   private def createTable(absoluteTableIdentifier: AbsoluteTableIdentifier): CarbonTable = {
     val tableInfo: TableInfo = new TableInfo()
-    tableInfo.setStorePath(absoluteTableIdentifier.getStorePath)
+    tableInfo.setTablePath(absoluteTableIdentifier.getTablePath)
     tableInfo.setDatabaseName(
       absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName)
     val tableSchema: TableSchema = new TableSchema()
@@ -293,7 +293,7 @@ object CarbonDataStoreCreator {
     tableInfo.setLastUpdatedTime(System.currentTimeMillis())
     tableInfo.setFactTable(tableSchema)
     val carbonTablePath: CarbonTablePath = CarbonStorePath.getCarbonTablePath(
-      absoluteTableIdentifier.getStorePath,
+      absoluteTableIdentifier.getTablePath,
       absoluteTableIdentifier.getCarbonTableIdentifier)
     val schemaFilePath: String = carbonTablePath.getSchemaFilePath
     val schemaMetadataPath: String =
@@ -351,22 +351,19 @@ object CarbonDataStoreCreator {
       line = reader.readLine()
     }
     val dictCache: Cache[DictionaryColumnUniqueIdentifier, ReverseDictionary] = CacheProvider
-      .getInstance.createCache(CacheType.REVERSE_DICTIONARY,
-      absoluteTableIdentifier.getStorePath)
+      .getInstance.createCache(CacheType.REVERSE_DICTIONARY)
     for (i <- set.indices) {
       val columnIdentifier: ColumnIdentifier =
         new ColumnIdentifier(dims.get(i).getColumnId, null, null)
       val dictionaryColumnUniqueIdentifier: DictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(
-          table.getCarbonTableIdentifier,
+          table.getAbsoluteTableIdentifier,
           columnIdentifier,
           columnIdentifier.getDataType,
-          CarbonStorePath.getCarbonTablePath(table.getStorePath,
+          CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier.getTablePath,
             table.getCarbonTableIdentifier)
         )
       val writer: CarbonDictionaryWriter = new CarbonDictionaryWriterImpl(
-        absoluteTableIdentifier.getStorePath,
-        absoluteTableIdentifier.getCarbonTableIdentifier,
         dictionaryColumnUniqueIdentifier)
       for (value <- set(i)) {
         writer.write(value)
@@ -376,10 +373,10 @@ object CarbonDataStoreCreator {
       val dict: Dictionary = dictCache
         .get(
           new DictionaryColumnUniqueIdentifier(
-            absoluteTableIdentifier.getCarbonTableIdentifier,
+            absoluteTableIdentifier,
             columnIdentifier,
             dims.get(i).getDataType,
-            CarbonStorePath.getCarbonTablePath(table.getStorePath,
+            CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier.getTablePath,
               table.getCarbonTableIdentifier)
           ))
         .asInstanceOf[Dictionary]
@@ -391,10 +388,7 @@ object CarbonDataStoreCreator {
           dict,
           dims.get(i).getDataType)
       val carbonDictionaryWriter: CarbonDictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(
-          absoluteTableIdentifier.getCarbonTableIdentifier,
-          dictionaryColumnUniqueIdentifier,
-          absoluteTableIdentifier.getStorePath)
+        new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier)
       try {
         carbonDictionaryWriter.writeSortIndex(dictionarySortInfo.getSortIndex)
         carbonDictionaryWriter.writeInvertedSortIndex(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CarbonV1toV3CompatabilityTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CarbonV1toV3CompatabilityTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CarbonV1toV3CompatabilityTestCase.scala
index 8115e27..f49e475 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CarbonV1toV3CompatabilityTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CarbonV1toV3CompatabilityTestCase.scala
@@ -52,7 +52,7 @@ class CarbonV1toV3CompatabilityTestCase extends QueryTest with BeforeAndAfterAll
     localspark.sessionState.asInstanceOf[CarbonSessionState].metadataHive
       .runSqlHive(
         s"ALTER TABLE default.t3 SET SERDEPROPERTIES" +
-        s"('tablePath'='$storeLocation/default/t3')")
+        s"('tablePath'='$storeLocation/default/t3', 'dbname'='default', 'tablename'='t3')")
     localspark.sql("show tables").show()
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
index 01146ee..2e26d7f 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
@@ -25,9 +25,10 @@ import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata}
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 
 /**
  * Test Class for AlterTableTestCase to verify all scenerios
@@ -106,12 +107,15 @@ class MergeIndexTestCase extends QueryTest with BeforeAndAfterAll {
   }
 
   private def getIndexFileCount(dbName: String, tableName: String, segment: String): Int = {
-    val identifier = AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sqlContext.sparkSession).storePath, dbName, tableName)
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable(dbName + "_" + tableName)
+    val identifier = carbonTable.getAbsoluteTableIdentifier
     val path = CarbonTablePath
       .getSegmentPath(identifier.getTablePath, segment)
     val carbonFiles = FileFactory.getCarbonFile(path).listFiles(new CarbonFileFilter {
-      override def accept(file: CarbonFile): Boolean = file.getName.endsWith(CarbonTablePath
-        .INDEX_FILE_EXT)
+      override def accept(file: CarbonFile): Boolean = {
+        file.getName.endsWith(CarbonTablePath
+          .INDEX_FILE_EXT)
+      }
     })
     if (carbonFiles != null) {
       carbonFiles.length

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/resources/dblocation/test.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/dblocation/test.csv b/integration/spark-common-test/src/test/resources/dblocation/test.csv
new file mode 100644
index 0000000..1dceb63
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/dblocation/test.csv
@@ -0,0 +1,6 @@
+c1,c2,c3,c5
+a,1,aa,aaa
+b,2,bb,bbb
+c,3,cc,ccc
+d,4,dd,ddd
+e,5,ee,eee

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
index 23d1292..a34f479 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
@@ -49,8 +49,8 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
       datbaseName: String,
       tableName: String): Boolean = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(datbaseName + "_" + tableName)
-    val partitionPath = CarbonStorePath.getCarbonTablePath(storeLocation,
-      carbonTable.getCarbonTableIdentifier).getPartitionDir("0")
+    val partitionPath = CarbonStorePath
+      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir("0")
     val fileType: FileFactory.FileType = FileFactory.getFileType(partitionPath)
     val carbonFile = FileFactory.getCarbonFile(partitionPath, fileType)
     val segments: ArrayBuffer[String] = ArrayBuffer()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
index 34981a2..ea1bbfd 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
@@ -24,6 +24,9 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.spark.sql.test.util.QueryTest
 
+import org.apache.carbondata.core.metadata.CarbonMetadata
+import org.apache.carbondata.core.util.path.CarbonStorePath
+
 class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
   var timeStampPropOrig: String = _
   override def beforeAll {
@@ -224,7 +227,10 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("insert overwrite table CarbonOverwrite select * from THive")
     sql("insert overwrite table HiveOverwrite select * from THive")
     checkAnswer(sql("select count(*) from CarbonOverwrite"), sql("select count(*) from HiveOverwrite"))
-    val folder = new File(s"$storeLocation/default/carbonoverwrite/Fact/Part0/")
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default" + "_" +"carbonoverwrite")
+    val partitionPath = CarbonStorePath
+      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir("0")
+    val folder = new File(partitionPath)
     assert(folder.isDirectory)
     assert(folder.list().length == 1)
   }
@@ -244,7 +250,11 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("LOAD DATA INPATH '" + resourcesPath + "/100_olap.csv' overwrite INTO table TCarbonSourceOverwrite options ('DELIMITER'=',', 'QUOTECHAR'='\', 'FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVe
 rsion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointDescription,gamePointId,contractNumber')")
     sql(s"LOAD DATA local INPATH '$resourcesPath/100_olap.csv' overwrite INTO TABLE HiveOverwrite")
     checkAnswer(sql("select count(*) from TCarbonSourceOverwrite"), sql("select count(*) from HiveOverwrite"))
-    val folder = new File(s"$storeLocation/default/tcarbonsourceoverwrite/Fact/Part0/")
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default" + "_" +"tcarbonsourceoverwrite")
+    val partitionPath = CarbonStorePath
+      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir("0")
+    val folder = new File(partitionPath)
+
     assert(folder.isDirectory)
     assert(folder.list().length == 1)
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
index c7b39ad..3873b0d 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
@@ -25,6 +25,7 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
+import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.util.CarbonProperties
 
 class CompactionSupportGlobalSortFunctionTest extends QueryTest with BeforeAndAfterEach with BeforeAndAfterAll {
@@ -527,7 +528,9 @@ class CompactionSupportGlobalSortFunctionTest extends QueryTest with BeforeAndAf
   }
 
   private def getIndexFileCount(tableName: String, segmentNo: String = "0"): Int = {
-    val store = storeLocation + "/default/" + tableName + "/Fact/Part0/Segment_" + segmentNo
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default" + "_" + tableName)
+    val store = carbonTable.getAbsoluteTableIdentifier.getTablePath + "/Fact/Part0/Segment_" +
+                segmentNo
     new SegmentIndexFileStore().getIndexFilesFromSegment(store).size()
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
index 78c835a..1de5c73 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
@@ -28,6 +28,9 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
 import org.apache.carbondata.core.util.path.CarbonTablePath
 
+import org.apache.carbondata.core.metadata.CarbonMetadata
+import org.apache.carbondata.core.util.path.CarbonStorePath
+
 class CompactionSupportGlobalSortParameterTest extends QueryTest with BeforeAndAfterEach with BeforeAndAfterAll {
   val filePath: String = s"$resourcesPath/globalsort"
   val file1: String = resourcesPath + "/globalsort/sample1.csv"
@@ -528,7 +531,9 @@ class CompactionSupportGlobalSortParameterTest extends QueryTest with BeforeAndA
   }
 
   private def getIndexFileCount(tableName: String, segmentNo: String = "0"): Int = {
-    val store = storeLocation + "/default/" + tableName + "/Fact/Part0/Segment_" + segmentNo
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default" + "_" + tableName)
+    val store = carbonTable.getAbsoluteTableIdentifier.getTablePath + "/Fact/Part0/Segment_" +
+                segmentNo
     new SegmentIndexFileStore().getIndexFilesFromSegment(store).size()
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
index 29f3492..508ca6c 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
@@ -42,14 +42,12 @@ class DataCompactionLockTest extends QueryTest with BeforeAndAfterAll {
           CarbonCommonConstants.DATABASE_DEFAULT_NAME, "compactionlocktesttable", "1")
       )
   val carbonTablePath: CarbonTablePath = CarbonStorePath
-    .getCarbonTablePath(absoluteTableIdentifier.getStorePath,
-      absoluteTableIdentifier.getCarbonTableIdentifier
-    )
+    .getCarbonTablePath(absoluteTableIdentifier)
   val dataPath: String = carbonTablePath.getMetadataDirectoryPath
 
   val carbonLock: ICarbonLock =
     CarbonLockFactory
-      .getCarbonLockObj(absoluteTableIdentifier.getCarbonTableIdentifier, LockUsage.COMPACTION_LOCK)
+      .getCarbonLockObj(absoluteTableIdentifier, LockUsage.COMPACTION_LOCK)
 
   override def beforeAll {
     CarbonProperties.getInstance()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
index 111ede7..9bf916e 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
@@ -20,7 +20,7 @@ import scala.collection.JavaConverters._
 
 import org.scalatest.BeforeAndAfterAll
 
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.datastore.TableSegmentUniqueIdentifier
 import org.apache.carbondata.core.datastore.block.SegmentTaskIndexWrapper
 import org.apache.carbondata.core.util.path.CarbonStorePath
@@ -76,12 +76,13 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
   test("delete merged folder and check segments") {
     // delete merged segments
     sql("clean files for table ignoremajor")
-    val identifier = new AbsoluteTableIdentifier(
-          CarbonProperties.getInstance.getProperty(CarbonCommonConstants.STORE_LOCATION),
-          new CarbonTableIdentifier(
-            CarbonCommonConstants.DATABASE_DEFAULT_NAME, "ignoremajor", "rrr")
-        )
-    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(identifier)
+
+    val carbonTable = CarbonMetadata.getInstance()
+      .getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME + "_" + "ignoremajor")
+    val absoluteTableIdentifier = carbonTable
+      .getAbsoluteTableIdentifier
+    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
+      absoluteTableIdentifier)
 
     // merged segment should not be there
     val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
@@ -89,9 +90,8 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
     assert(segments.contains("2.1"))
     assert(!segments.contains("2"))
     assert(!segments.contains("3"))
-    val cacheClient = new CacheClient(CarbonProperties.getInstance.
-      getProperty(CarbonCommonConstants.STORE_LOCATION));
-    val segmentIdentifier = new TableSegmentUniqueIdentifier(identifier, "2")
+    val cacheClient = new CacheClient();
+    val segmentIdentifier = new TableSegmentUniqueIdentifier(absoluteTableIdentifier, "2")
     val wrapper: SegmentTaskIndexWrapper = cacheClient.getSegmentAccessClient.
       getIfPresent(segmentIdentifier)
     assert(null == wrapper)
@@ -109,12 +109,12 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
     catch {
       case _:Throwable => assert(true)
     }
-    val carbontablePath = CarbonStorePath
-      .getCarbonTablePath(CarbonProperties.getInstance
-        .getProperty(CarbonCommonConstants.STORE_LOCATION),
-        new CarbonTableIdentifier(
-          CarbonCommonConstants.DATABASE_DEFAULT_NAME, "ignoremajor", "rrr")
-      )
+
+    val carbonTable = CarbonMetadata.getInstance()
+      .getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME + "_" + "ignoremajor")
+    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+
+    val carbontablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
       .getMetadataDirectoryPath
     val segs = SegmentStatusManager.readLoadMetadata(carbontablePath)
 
@@ -130,13 +130,11 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
       "delete from table ignoremajor where segment.starttime before " +
         " '2222-01-01 19:35:01'"
     )
+    val carbonTable = CarbonMetadata.getInstance()
+      .getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME + "_" + "ignoremajor")
+    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
     val carbontablePath = CarbonStorePath
-      .getCarbonTablePath(CarbonProperties.getInstance
-        .getProperty(CarbonCommonConstants.STORE_LOCATION),
-        new CarbonTableIdentifier(
-          CarbonCommonConstants.DATABASE_DEFAULT_NAME, "ignoremajor", "rrr")
-      )
-      .getMetadataDirectoryPath
+      .getCarbonTablePath(absoluteTableIdentifier).getMetadataDirectoryPath
     val segs = SegmentStatusManager.readLoadMetadata(carbontablePath)
 
     // status should remain as compacted for segment 2.
@@ -171,12 +169,12 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
         "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
     sql("alter table testmajor compact 'major'")
-    val identifier = new AbsoluteTableIdentifier(
-      CarbonProperties.getInstance.getProperty(CarbonCommonConstants.STORE_LOCATION),
-      new CarbonTableIdentifier(
-        CarbonCommonConstants.DATABASE_DEFAULT_NAME, "testmajor", "ttt")
-    )
-    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(identifier)
+
+    val carbonTable = CarbonMetadata.getInstance()
+      .getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME + "_" + "testmajor")
+    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
+      absoluteTableIdentifier)
 
     // merged segment should not be there
     val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala
index 4976c24..02560d9 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala
@@ -19,7 +19,8 @@ package org.apache.carbondata.spark.testsuite.datacompaction
 import scala.collection.JavaConverters._
 
 import org.scalatest.BeforeAndAfterAll
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.CarbonProperties
@@ -77,14 +78,12 @@ class MajorCompactionStopsAfterCompaction extends QueryTest with BeforeAndAfterA
     var status = false
     var noOfRetries = 0
     while (!status && noOfRetries < 10) {
+      val carbonTable = CarbonMetadata.getInstance()
+        .getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME + "_" + "stopmajor")
+      val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
 
-      val identifier = new AbsoluteTableIdentifier(
-            CarbonProperties.getInstance.getProperty(CarbonCommonConstants.STORE_LOCATION),
-            new CarbonTableIdentifier(
-              CarbonCommonConstants.DATABASE_DEFAULT_NAME, "stopmajor", noOfRetries + "")
-          )
-
-      val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(identifier)
+      val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
+        absoluteTableIdentifier)
 
       val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
 //      segments.foreach(seg =>
@@ -111,12 +110,12 @@ class MajorCompactionStopsAfterCompaction extends QueryTest with BeforeAndAfterA
     // delete merged segments
     sql("clean files for table stopmajor")
 
-    val identifier = new AbsoluteTableIdentifier(
-          CarbonProperties.getInstance.getProperty(CarbonCommonConstants.STORE_LOCATION),
-          new CarbonTableIdentifier(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "stopmajor", "rrr")
-        )
+    val carbonTable = CarbonMetadata.getInstance()
+      .getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME + "_" + "stopmajor")
+    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
 
-    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(identifier)
+    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
+      absoluteTableIdentifier)
 
     // merged segment should not be there
     val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
index cb3da40..ae25894 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
@@ -27,6 +27,8 @@ import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.spark.sql.test.util.QueryTest
 
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
+import org.apache.carbondata.core.metadata.CarbonMetadata
+import org.apache.carbondata.core.util.path.CarbonStorePath
 
 class TestBatchSortDataLoad extends QueryTest with BeforeAndAfterAll {
   var filePath: String = _
@@ -186,8 +188,12 @@ class TestBatchSortDataLoad extends QueryTest with BeforeAndAfterAll {
   }
 
   def getIndexfileCount(tableName: String, segmentNo: String = "0"): Int = {
-    val store  = storeLocation +"/default/"+ tableName + "/Fact/Part0/Segment_"+segmentNo
-    new SegmentIndexFileStore().getIndexFilesFromSegment(store).size()
+    val carbonTable = CarbonMetadata.getInstance()
+      .getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME + "_" + tableName)
+    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath("0", segmentNo)
+    new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
   }
 
   override def afterAll {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
index c4152a1..44bb2dd 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
@@ -22,10 +22,12 @@ import java.io.{File, FilenameFilter}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.reader.CarbonIndexFileReader
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.core.metadata.CarbonMetadata
+
 class TestDataLoadWithFileName extends QueryTest with BeforeAndAfterAll {
   var originVersion = ""
 
@@ -45,7 +47,10 @@ class TestDataLoadWithFileName extends QueryTest with BeforeAndAfterAll {
     val testData = s"$resourcesPath/sample.csv"
     sql(s"LOAD DATA LOCAL INPATH '$testData' into table test_table_v3")
     val indexReader = new CarbonIndexFileReader()
-    val carbonIndexPaths = new File(s"$storeLocation/default/test_table_v3/Fact/Part0/Segment_0/")
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_test_table_v3")
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath("0", "0")
+    val carbonIndexPaths = new File(segmentDir)
       .listFiles(new FilenameFilter {
         override def accept(dir: File, name: String): Boolean = {
           name.endsWith(CarbonTablePath.getCarbonIndexExtension)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
index 5d0c055..9f941f0 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
@@ -29,6 +29,9 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
 
+import org.apache.carbondata.core.metadata.CarbonMetadata
+import org.apache.carbondata.core.util.path.CarbonStorePath
+
 class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with BeforeAndAfterAll {
   var filePath: String = s"$resourcesPath/globalsort"
 
@@ -236,11 +239,13 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
         | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('SORT_SCOPE'='GLOBAL_SORT')
       """.stripMargin)
     sql(s"LOAD DATA LOCAL INPATH '$filePath' INTO TABLE carbon_localsort_update")
-    sql("UPDATE carbon_localsort_update SET (name) = ('bb') WHERE id = 2").show
 
+    sql("UPDATE carbon_localsort_update SET (name) = ('bb') WHERE id = 2").show
+    sql("select * from carbon_localsort_update").show()
     sql(s"LOAD DATA LOCAL INPATH '$filePath' INTO TABLE carbon_globalsort")
+    sql("select * from carbon_globalsort").show()
     sql("UPDATE carbon_globalsort SET (name) = ('bb') WHERE id = 2").show
-
+    sql("select * from carbon_globalsort").show()
     checkAnswer(sql("SELECT COUNT(*) FROM carbon_globalsort"), Seq(Row(12)))
     checkAnswer(sql("SELECT name FROM carbon_globalsort WHERE id = 2"), Seq(Row("bb")))
     checkAnswer(sql("SELECT * FROM carbon_globalsort ORDER BY name"),
@@ -326,7 +331,9 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
   }
 
   private def getIndexFileCount(tableName: String, segmentNo: String = "0"): Int = {
-    val store  = storeLocation + "/default/" + tableName + "/Fact/Part0/Segment_" + segmentNo
-    new SegmentIndexFileStore().getIndexFilesFromSegment(store).size()
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default"+"_"+tableName)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath("0", segmentNo)
+    new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
   }
 }


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

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
index 24996ed..357a812 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
@@ -90,18 +90,17 @@ trait CarbonMetaStore {
    *
    * @param carbonTableIdentifier
    * @param thriftTableInfo
-   * @param tablePath
+   * @param absoluteTableIdentifier
    * @param sparkSession
    */
   def revertTableSchemaInAlterFailure(carbonTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
-      tablePath: String)
+      absoluteTableIdentifier: AbsoluteTableIdentifier)
     (sparkSession: SparkSession): String
 
 
-  def revertTableSchemaForPreAggCreationFailure(carbonTableIdentifier: CarbonTableIdentifier,
-      thriftTableInfo: org.apache.carbondata.format.TableInfo,
-      tablePath: String)(sparkSession: SparkSession): String
+  def revertTableSchemaForPreAggCreationFailure(absoluteTableIdentifier: AbsoluteTableIdentifier,
+      thriftTableInfo: org.apache.carbondata.format.TableInfo)(sparkSession: SparkSession): String
   /**
    * Prepare Thrift Schema from wrapper TableInfo and write to disk
    */
@@ -113,7 +112,7 @@ trait CarbonMetaStore {
    * @return
    */
   def generateTableSchemaString(tableInfo: schema.table.TableInfo,
-      tablePath: String): String
+      absoluteTableIdentifier: AbsoluteTableIdentifier): String
 
   /**
    * This method will remove the table meta from catalog metadata array
@@ -128,12 +127,12 @@ trait CarbonMetaStore {
 
   def isTablePathExists(tableIdentifier: TableIdentifier)(sparkSession: SparkSession): Boolean
 
-  def dropTable(tablePath: String, tableIdentifier: TableIdentifier)
+  def dropTable(tableIdentifier: AbsoluteTableIdentifier)
     (sparkSession: SparkSession)
 
-  def updateAndTouchSchemasUpdatedTime(basePath: String)
+  def updateAndTouchSchemasUpdatedTime()
 
-  def checkSchemasModifiedTimeAndReloadTables(storePath: String)
+  def checkSchemasModifiedTimeAndReloadTables()
 
   def isReadFromHiveMetaStore : Boolean
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
index 825f6ed..e587395 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -108,7 +108,7 @@ class CarbonSessionCatalog(
     var isRefreshed = false
     val storePath = CarbonEnv.getInstance(sparkSession).storePath
     carbonEnv.carbonMetastore.
-      checkSchemasModifiedTimeAndReloadTables(storePath)
+      checkSchemasModifiedTimeAndReloadTables()
 
     val tableMeta = carbonEnv.carbonMetastore
       .getTableFromMetadataCache(carbonDatasourceHadoopRelation.carbonTable.getDatabaseName,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
index 7d25efd..b76b24e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
@@ -17,8 +17,9 @@
 
 package org.apache.spark.sql.hive.execution.command
 
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
 import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.execution.command._
 
@@ -38,15 +39,23 @@ case class CarbonDropDatabaseCommand(command: DropDatabaseCommand)
     if (sparkSession.sessionState.catalog.listDatabases().exists(_.equalsIgnoreCase(dbName))) {
       tablesInDB = sparkSession.sessionState.catalog.listTables(dbName)
     }
+    var databaseLocation = ""
+    try {
+      databaseLocation = GetDB.getDatabaseLocation(dbName, sparkSession,
+        CarbonEnv.getInstance(sparkSession).storePath)
+    } catch {
+      case e: NoSuchDatabaseException =>
+        // ignore the exception as exception will be handled by hive command.run
+      databaseLocation = CarbonEnv.getInstance(sparkSession).storePath
+    }
     // DropHiveDB command will fail if cascade is false and one or more table exists in database
-    val rows = command.run(sparkSession)
     if (command.cascade && tablesInDB != null) {
       tablesInDB.foreach { tableName =>
         CarbonDropTableCommand(true, tableName.database, tableName.table).run(sparkSession)
       }
     }
-    CarbonUtil.dropDatabaseDirectory(dbName.toLowerCase,
-      CarbonEnv.getInstance(sparkSession).storePath)
+    CarbonUtil.dropDatabaseDirectory(dbName.toLowerCase, databaseLocation)
+    val rows = command.run(sparkSession)
     rows
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index bda4eeb..153b169 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -21,6 +21,7 @@ import scala.collection.JavaConverters._
 import scala.collection.mutable
 import scala.collection.mutable.ListBuffer
 
+import org.apache.hadoop.fs.Path
 import org.apache.spark.SparkConf
 import org.apache.spark.sql.{CarbonEnv, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
@@ -30,14 +31,13 @@ import org.apache.spark.sql.hive.HiveExternalCatalog._
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock}
-import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier
+import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
-import org.apache.carbondata.format.TableInfo
 
 object AlterTableUtil {
 
@@ -69,7 +69,7 @@ object AlterTableUtil {
     val acquiredLocks = ListBuffer[ICarbonLock]()
     try {
       locksToBeAcquired.foreach { lock =>
-        acquiredLocks += CarbonLockUtil.getLockObject(table.getCarbonTableIdentifier, lock)
+        acquiredLocks += CarbonLockUtil.getLockObject(table.getAbsoluteTableIdentifier, lock)
       }
       acquiredLocks.toList
     } catch {
@@ -102,15 +102,14 @@ object AlterTableUtil {
    * @param locksAcquired
    * @param dbName
    * @param tableName
-   * @param storeLocation
+   * @param tablePath
    */
   def releaseLocksManually(locks: List[ICarbonLock],
       locksAcquired: List[String],
       dbName: String,
       tableName: String,
-      storeLocation: String): Unit = {
-    val lockLocation = storeLocation + CarbonCommonConstants.FILE_SEPARATOR +
-                       dbName + CarbonCommonConstants.FILE_SEPARATOR + tableName
+      tablePath: String): Unit = {
+    val lockLocation = tablePath
     locks.zip(locksAcquired).foreach { case (carbonLock, lockType) =>
       val lockFilePath = lockLocation + CarbonCommonConstants.FILE_SEPARATOR +
                          lockType
@@ -185,20 +184,23 @@ object AlterTableUtil {
    */
   def revertRenameTableChanges(oldTableIdentifier: TableIdentifier,
       newTableName: String,
-      storePath: String,
+      tablePath: String,
       tableId: String,
       timeStamp: Long)
     (sparkSession: SparkSession): Unit = {
     val database = oldTableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase)
+    val oldCarbonTableIdentifier = new CarbonTableIdentifier(database,
+      oldTableIdentifier.table, tableId)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(tablePath, oldCarbonTableIdentifier)
     val newCarbonTableIdentifier = new CarbonTableIdentifier(database, newTableName, tableId)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, newCarbonTableIdentifier)
+    val newTablePath = CarbonUtil.getNewTablePath(new Path(tablePath), newCarbonTableIdentifier)
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val tableMetadataFile = carbonTablePath.getPath
     val fileType = FileFactory.getFileType(tableMetadataFile)
     if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
       val tableInfo = if (metastore.isReadFromHiveMetaStore) {
         // In case of hive metastore we first update the carbonschema inside old table only.
-        metastore.getThriftTableInfo(CarbonStorePath.getCarbonTablePath(storePath,
+        metastore.getThriftTableInfo(CarbonStorePath.getCarbonTablePath(tablePath,
           new CarbonTableIdentifier(database, oldTableIdentifier.table, tableId)))(sparkSession)
       } else {
         metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
@@ -210,9 +212,10 @@ object AlterTableUtil {
         FileFactory.getCarbonFile(carbonTablePath.getPath, fileType)
           .renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
                        oldTableIdentifier.table)
-        val tableIdentifier = new CarbonTableIdentifier(database, oldTableIdentifier.table, tableId)
-        metastore.revertTableSchemaInAlterFailure(tableIdentifier,
-          tableInfo, carbonTablePath.getPath)(sparkSession)
+        val absoluteTableIdentifier = new AbsoluteTableIdentifier(newTablePath,
+          newCarbonTableIdentifier)
+        metastore.revertTableSchemaInAlterFailure(oldCarbonTableIdentifier,
+          tableInfo, absoluteTableIdentifier)(sparkSession)
         metastore.removeTableFromMetadata(database, newTableName)
       }
     }
@@ -233,7 +236,7 @@ object AlterTableUtil {
       .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation].tableMeta
       .carbonTable
 
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
       carbonTable.getCarbonTableIdentifier)
     val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
@@ -244,7 +247,7 @@ object AlterTableUtil {
       thriftTable.fact_table.table_columns.removeAll(addedSchemas)
       metastore
         .revertTableSchemaInAlterFailure(carbonTable.getCarbonTableIdentifier,
-          thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+          thriftTable, carbonTable.getAbsoluteTableIdentifier)(sparkSession)
     }
   }
 
@@ -262,7 +265,7 @@ object AlterTableUtil {
     val carbonTable = metastore
       .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation].tableMeta
       .carbonTable
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
       carbonTable.getCarbonTableIdentifier)
     val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
@@ -279,7 +282,7 @@ object AlterTableUtil {
       }
       metastore
         .revertTableSchemaInAlterFailure(carbonTable.getCarbonTableIdentifier,
-          thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+          thriftTable, carbonTable.getAbsoluteTableIdentifier)(sparkSession)
     }
   }
 
@@ -297,7 +300,7 @@ object AlterTableUtil {
     val carbonTable = metastore
       .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation].tableMeta
       .carbonTable
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
       carbonTable.getCarbonTableIdentifier)
     val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
@@ -317,7 +320,7 @@ object AlterTableUtil {
       }
       metastore
         .revertTableSchemaInAlterFailure(carbonTable.getCarbonTableIdentifier,
-          thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+          thriftTable, carbonTable.getAbsoluteTableIdentifier)(sparkSession)
     }
   }
 
@@ -351,7 +354,7 @@ object AlterTableUtil {
         .tableMeta.carbonTable
       // get the latest carbon table
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
         carbonTable.getCarbonTableIdentifier)
       val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
@@ -359,7 +362,7 @@ object AlterTableUtil {
         .fromExternalToWrapperTableInfo(thriftTableInfo,
           dbName,
           tableName,
-          carbonTable.getStorePath)
+          carbonTable.getTablePath)
       val schemaEvolutionEntry = new org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
       schemaEvolutionEntry.setTimeStamp(timeStamp)
       val thriftTable = schemaConverter

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/util/CleanFiles.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/CleanFiles.scala b/integration/spark2/src/main/scala/org/apache/spark/util/CleanFiles.scala
index 421cd2e..dcfbaea 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/CleanFiles.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/CleanFiles.scala
@@ -61,7 +61,7 @@ object CleanFiles {
     }
     val spark = TableAPIUtil.spark(storePath, s"CleanFiles: $dbName.$tableName")
     CarbonEnv.getInstance(spark).carbonMetastore.
-      checkSchemasModifiedTimeAndReloadTables(CarbonEnv.getInstance(spark).storePath)
+      checkSchemasModifiedTimeAndReloadTables()
     cleanFiles(spark, dbName, tableName, storePath, forceTableClean)
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala b/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
index 91121ce..709f474 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
@@ -58,7 +58,7 @@ object Compaction {
     val compactionType = TableAPIUtil.escape(args(2))
     val spark = TableAPIUtil.spark(storePath, s"Compaction: $dbName.$tableName")
     CarbonEnv.getInstance(spark).carbonMetastore.
-      checkSchemasModifiedTimeAndReloadTables(CarbonEnv.getInstance(spark).storePath)
+      checkSchemasModifiedTimeAndReloadTables()
     compaction(spark, dbName, tableName, compactionType)
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala b/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala
index 4aaec8f..8375762 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala
@@ -48,7 +48,7 @@ object DeleteSegmentByDate {
     val dateValue = TableAPIUtil.escape(args(2))
     val spark = TableAPIUtil.spark(storePath, s"DeleteSegmentByDate: $dbName.$tableName")
     CarbonEnv.getInstance(spark).carbonMetastore.
-      checkSchemasModifiedTimeAndReloadTables(CarbonEnv.getInstance(spark).storePath)
+      checkSchemasModifiedTimeAndReloadTables()
     deleteSegmentByDate(spark, dbName, tableName, dateValue)
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala b/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala
index c86c7f5..9b87504 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala
@@ -53,7 +53,7 @@ object DeleteSegmentById {
     val segmentIds = extractSegmentIds(TableAPIUtil.escape(args(2)))
     val spark = TableAPIUtil.spark(storePath, s"DeleteSegmentById: $dbName.$tableName")
     CarbonEnv.getInstance(spark).carbonMetastore.
-      checkSchemasModifiedTimeAndReloadTables(CarbonEnv.getInstance(spark).storePath)
+      checkSchemasModifiedTimeAndReloadTables()
     deleteSegmentById(spark, dbName, tableName, segmentIds)
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala b/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
index 501402b..13883ac 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
@@ -81,7 +81,7 @@ object TableLoader {
     val spark = TableAPIUtil.spark(storePath, s"TableLoader: $dbName.$tableName")
 
     CarbonEnv.getInstance(spark).carbonMetastore.
-      checkSchemasModifiedTimeAndReloadTables(CarbonEnv.getInstance(spark).storePath)
+      checkSchemasModifiedTimeAndReloadTables()
     loadTable(spark, Option(dbName), tableName, inputPaths, map)
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
index 04de9a3..23cba20 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
@@ -843,8 +843,8 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
   }
 
   def getDataFiles(carbonTable: CarbonTable, segmentId: String): Array[CarbonFile] = {
-    val tablePath = new CarbonTablePath(carbonTable.getStorePath, carbonTable.getDatabaseName,
-      carbonTable.getFactTableName)
+    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
+      carbonTable.getTablePath)
     val segmentDir = tablePath.getCarbonDataDirectoryPath("0", segmentId)
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
index 5c7d451..a3024be 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
@@ -64,7 +64,7 @@ class AllDictionaryTestCase extends Spark2QueryTest with BeforeAndAfterAll {
     carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
     // Create table and metadata folders if not exist
     val carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(table.getStorePath, table.getCarbonTableIdentifier)
+      .getCarbonTablePath(table.getTablePath, table.getCarbonTableIdentifier)
     val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
     val fileType = FileFactory.getFileType(metadataDirectoryPath)
     if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
index fd3b2cd..930de43 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.test.TestQueryExecutor
 
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
 
@@ -41,11 +41,12 @@ object DictionaryTestCaseUtil {
     val table = relation.tableMeta.carbonTable
     val dimension = table.getDimensionByName(table.getFactTableName, columnName)
     val tableIdentifier = new CarbonTableIdentifier(table.getDatabaseName, table.getFactTableName, "uniqueid")
-    val columnIdentifier = new DictionaryColumnUniqueIdentifier(tableIdentifier,
+    val  absoluteTableIdentifier = new AbsoluteTableIdentifier(table.getTablePath, tableIdentifier)
+    val columnIdentifier = new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
       dimension.getColumnIdentifier, dimension.getDataType,
       CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier)
     )
-    val dict = CarbonLoaderUtil.getDictionary(columnIdentifier, TestQueryExecutor.storeLocation)
+    val dict = CarbonLoaderUtil.getDictionary(columnIdentifier)
     assert(dict.getSurrogateKey(value) != CarbonCommonConstants.INVALID_SURROGATE_KEY)
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
index 399665f..d37a68b 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
@@ -177,7 +177,7 @@ class ExternalColumnDictionaryTestCase extends Spark2QueryTest with BeforeAndAft
     carbonLoadModel.setMaxColumns("100")
     // Create table and metadata folders if not exist
     val carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(table.getStorePath, table.getCarbonTableIdentifier)
+      .getCarbonTablePath(table.getTablePath, table.getCarbonTableIdentifier)
     val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
     val fileType = FileFactory.getFileType(metadataDirectoryPath)
     if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
index deb6287..3fb1424 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -196,7 +196,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
       thread1.start()
       // use thread pool to catch the exception of sink thread
       val pool = Executors.newSingleThreadExecutor()
-      val thread2 = createSocketStreamingThread(spark, tablePath)
+      val thread2 = createSocketStreamingThread(spark, tablePath, identifier)
       val future = pool.submit(thread2)
       Thread.sleep(1000)
       thread1.interrupt()
@@ -242,7 +242,8 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     val csvDataDir = new File("target/csvdata").getCanonicalPath
     // streaming ingest 10 rows
     generateCSVDataFile(spark, idStart = 10, rowNums = 10, csvDataDir)
-    val thread = createFileStreamingThread(spark, tablePath, csvDataDir, intervalSecond = 1)
+    val thread = createFileStreamingThread(spark, tablePath, csvDataDir, intervalSecond = 1,
+      identifier )
     thread.start()
     Thread.sleep(2000)
     generateCSVDataFile(spark, idStart = 30, rowNums = 10, csvDataDir)
@@ -636,6 +637,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
   def createSocketStreamingThread(
       spark: SparkSession,
       tablePath: CarbonTablePath,
+      tableIdentifier: TableIdentifier,
       badRecordAction: String = "force",
       intervalSecond: Int = 2,
       handoffSize: Long = CarbonStreamOutputFormat.HANDOFF_SIZE_DEFAULT): Thread = {
@@ -656,6 +658,8 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
             .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
             .option("tablePath", tablePath.getPath)
             .option("bad_records_action", badRecordAction)
+            .option("dbName", tableIdentifier.database.get)
+            .option("tableName", tableIdentifier.table)
             .option(CarbonStreamOutputFormat.HANDOFF_SIZE, handoffSize)
             .start()
           qry.awaitTermination()
@@ -698,7 +702,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
         badRecords = generateBadRecords)
       val thread2 = createSocketStreamingThread(
         spark = spark,
-        tablePath = tablePath,
+        tablePath = tablePath, identifier,
         badRecordAction = badRecordAction,
         intervalSecond = intervalOfIngest,
         handoffSize = handoffSize)
@@ -740,7 +744,8 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
       spark: SparkSession,
       tablePath: CarbonTablePath,
       csvDataDir: String,
-      intervalSecond: Int): Thread = {
+      intervalSecond: Int,
+      tableIdentifier: TableIdentifier): Thread = {
     new Thread() {
       override def run(): Unit = {
         val inputSchema = new StructType()
@@ -765,6 +770,8 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
             .trigger(ProcessingTime(s"${ intervalSecond } seconds"))
             .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
             .option("tablePath", tablePath.getPath)
+            .option("dbName", tableIdentifier.database.get)
+            .option("tableName", tableIdentifier.table)
             .start()
 
           qry.awaitTermination()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
index e574d7f..3b6b85d 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
@@ -437,7 +437,7 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
   test("test to check if the lock file is successfully deleted") {
       sql("create table lock_check(id int, name string) stored by 'carbondata'")
     sql("alter table lock_check rename to lock_rename")
-    assert(!new File(s"${ CarbonCommonConstants.STORE_LOCATION } + /default/lock_rename/meta.lock")
+    assert(!new File(s"${ CarbonCommonConstants.STORE_LOCATION } + /lock_rename/meta.lock")
       .exists())
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
index adf4574..9f89226 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
@@ -26,7 +26,8 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.api.CarbonStore
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.metadata.CarbonMetadata
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 
 class CarbonCommandSuite extends Spark2QueryTest with BeforeAndAfterAll {
 
@@ -115,7 +116,8 @@ class CarbonCommandSuite extends Spark2QueryTest with BeforeAndAfterAll {
     createAndLoadTestTable(table, "csv_table")
     DeleteSegmentById.main(Array(s"${location}", table, "0"))
     CleanFiles.main(Array(s"${location}", table))
-    val tablePath = s"${location}${File.separator}default${File.separator}$table"
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_"+table)
+    val tablePath = carbonTable.getAbsoluteTableIdentifier.getTablePath
     val f = new File(s"$tablePath/Fact/Part0")
     assert(f.isDirectory)
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
index 95d7d2e..f70e38e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
@@ -38,7 +38,7 @@ import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -122,16 +122,16 @@ public class PrimitiveDataType implements GenericDataType<Object> {
    */
   public PrimitiveDataType(String name, String parentname, String columnId,
       CarbonDimension carbonDimension, Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
-      CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
-      String storePath, Map<Object, Integer> localCache) {
+      AbsoluteTableIdentifier absoluteTableIdentifier, DictionaryClient client, Boolean useOnePass,
+      Map<Object, Integer> localCache) {
     this.name = name;
     this.parentname = parentname;
     this.columnId = columnId;
     this.carbonDimension = carbonDimension;
     DictionaryColumnUniqueIdentifier identifier =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
             carbonDimension.getColumnIdentifier(), carbonDimension.getDataType(),
-            CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier));
+            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
     try {
       if (carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
         dictionaryGenerator = new DirectDictionary(DirectDictionaryKeyGeneratorFactory
@@ -139,13 +139,14 @@ public class PrimitiveDataType implements GenericDataType<Object> {
       } else {
         Dictionary dictionary = null;
         if (useOnePass) {
-          if (CarbonUtil.isFileExistsForGivenColumn(storePath, identifier)) {
+          if (CarbonUtil.isFileExistsForGivenColumn(identifier)) {
             dictionary = cache.get(identifier);
           }
           DictionaryMessage dictionaryMessage = new DictionaryMessage();
           dictionaryMessage.setColumnName(carbonDimension.getColName());
           // for table initialization
-          dictionaryMessage.setTableUniqueId(carbonTableIdentifier.getTableId());
+          dictionaryMessage
+              .setTableUniqueId(absoluteTableIdentifier.getCarbonTableIdentifier().getTableId());
           dictionaryMessage.setData("0");
           // for generate dictionary
           dictionaryMessage.setType(DictionaryMessageType.DICT_GENERATION);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
index 05104a2..442d93e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
@@ -142,7 +142,7 @@ public final class DataLoadProcessBuilder {
     CarbonProperties.getInstance().addProperty(tempLocationKey,
         StringUtils.join(storeLocation, File.pathSeparator));
     CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS, loadModel.getStorePath());
+        .addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS, loadModel.getTablePath());
 
     return createConfiguration(loadModel);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
index 7045101..4ac8850 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
@@ -33,7 +33,7 @@ import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.dictionary.client.DictionaryClient;
 import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
 import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
@@ -65,8 +65,8 @@ public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConvert
 
   public DictionaryFieldConverterImpl(DataField dataField,
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
-      CarbonTableIdentifier carbonTableIdentifier, String nullFormat, int index,
-      DictionaryClient client, boolean useOnePass, String storePath,
+      AbsoluteTableIdentifier absoluteTableIdentifier, String nullFormat, int index,
+      DictionaryClient client, boolean useOnePass,
       Map<Object, Integer> localCache, boolean isEmptyBadRecord,
       DictionaryColumnUniqueIdentifier identifier) throws IOException {
     this.index = index;
@@ -76,13 +76,14 @@ public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConvert
 
     // if use one pass, use DictionaryServerClientDictionary
     if (useOnePass) {
-      if (CarbonUtil.isFileExistsForGivenColumn(storePath, identifier)) {
+      if (CarbonUtil.isFileExistsForGivenColumn(identifier)) {
         dictionary = cache.get(identifier);
       }
       dictionaryMessage = new DictionaryMessage();
       dictionaryMessage.setColumnName(dataField.getColumn().getColName());
       // for table initialization
-      dictionaryMessage.setTableUniqueId(carbonTableIdentifier.getTableId());
+      dictionaryMessage
+          .setTableUniqueId(absoluteTableIdentifier.getCarbonTableIdentifier().getTableId());
       dictionaryMessage.setData("0");
       // for generate dictionary
       dictionaryMessage.setType(DictionaryMessageType.DICT_GENERATION);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
index 22d15d9..778e1b3 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
@@ -24,6 +24,7 @@ import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.apache.carbondata.core.dictionary.client.DictionaryClient;
+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;
@@ -33,7 +34,9 @@ import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
+import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.datatypes.ArrayDataType;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
 import org.apache.carbondata.processing.datatypes.PrimitiveDataType;
@@ -61,15 +64,15 @@ public class FieldEncoderFactory {
    *
    * @param dataField             column schema
    * @param cache                 dicionary cache.
-   * @param carbonTableIdentifier table identifier
+   * @param absoluteTableIdentifier table identifier
    * @param index                 index of column in the row.
    * @param isEmptyBadRecord
    * @return
    */
   public FieldConverter createFieldEncoder(DataField dataField,
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
-      CarbonTableIdentifier carbonTableIdentifier, int index, String nullFormat,
-      DictionaryClient client, Boolean useOnePass, String storePath,
+      AbsoluteTableIdentifier absoluteTableIdentifier, int index, String nullFormat,
+      DictionaryClient client, Boolean useOnePass,
       Map<Object, Integer> localCache, boolean isEmptyBadRecord)
       throws IOException {
     // Converters are only needed for dimensions and measures it return null.
@@ -85,11 +88,11 @@ public class FieldEncoderFactory {
         // in case of child table it will use parent table dictionary
         if (null == dataField.getColumn().getColumnSchema().getParentColumnTableRelations()
             || dataField.getColumn().getColumnSchema().getParentColumnTableRelations().isEmpty()) {
-          identifier = new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
+          identifier = new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
               dataField.getColumn().getColumnIdentifier(), dataField.getColumn().getDataType(),
-              CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier));
-          return new DictionaryFieldConverterImpl(dataField, cache, carbonTableIdentifier,
-              nullFormat, index, client, useOnePass, storePath, localCache, isEmptyBadRecord,
+              CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
+          return new DictionaryFieldConverterImpl(dataField, cache, absoluteTableIdentifier,
+              nullFormat, index, client, useOnePass, localCache, isEmptyBadRecord,
               identifier);
         } else {
           ParentColumnTableRelation parentColumnTableRelation =
@@ -103,17 +106,21 @@ public class FieldEncoderFactory {
           ColumnIdentifier parentColumnIdentifier =
               new ColumnIdentifier(parentColumnTableRelation.getColumnId(), null,
                   dataField.getColumn().getDataType());
-          identifier =
-              new DictionaryColumnUniqueIdentifier(parentTableIdentifier, parentColumnIdentifier,
-                  dataField.getColumn().getDataType(),
-                  CarbonStorePath.getCarbonTablePath(storePath, parentTableIdentifier));
-          return new DictionaryFieldConverterImpl(dataField, cache, parentTableIdentifier,
-              nullFormat, index, null, false, storePath, null, isEmptyBadRecord, identifier);
+          CarbonTablePath carbonTablePath =
+              CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
+          AbsoluteTableIdentifier parentAbsoluteTableIdentifier = new AbsoluteTableIdentifier(
+              CarbonUtil.getNewTablePath(carbonTablePath, parentTableIdentifier),
+              parentTableIdentifier);
+          identifier = new DictionaryColumnUniqueIdentifier(parentAbsoluteTableIdentifier,
+              parentColumnIdentifier, dataField.getColumn().getDataType(),
+              CarbonStorePath.getCarbonTablePath(parentAbsoluteTableIdentifier));
+          return new DictionaryFieldConverterImpl(dataField, cache, parentAbsoluteTableIdentifier,
+              nullFormat, index, null, false, null, isEmptyBadRecord, identifier);
         }
       } else if (dataField.getColumn().isComplex()) {
         return new ComplexFieldConverterImpl(
-            createComplexType(dataField, cache, carbonTableIdentifier,
-                client, useOnePass, storePath, localCache), index);
+            createComplexType(dataField, cache, absoluteTableIdentifier,
+                client, useOnePass, localCache), index);
       } else {
         return new NonDictionaryFieldConverterImpl(dataField, nullFormat, index, isEmptyBadRecord);
       }
@@ -127,10 +134,10 @@ public class FieldEncoderFactory {
    */
   private static GenericDataType createComplexType(DataField dataField,
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
-      CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
-      String storePath, Map<Object, Integer> localCache) {
+      AbsoluteTableIdentifier absoluteTableIdentifier, DictionaryClient client, Boolean useOnePass,
+      Map<Object, Integer> localCache) {
     return createComplexType(dataField.getColumn(), dataField.getColumn().getColName(), cache,
-        carbonTableIdentifier, client, useOnePass, storePath, localCache);
+        absoluteTableIdentifier, client, useOnePass, localCache);
   }
 
   /**
@@ -140,8 +147,8 @@ public class FieldEncoderFactory {
    */
   private static GenericDataType createComplexType(CarbonColumn carbonColumn, String parentName,
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
-      CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
-      String storePath, Map<Object, Integer> localCache) {
+      AbsoluteTableIdentifier absoluteTableIdentifier, DictionaryClient client, Boolean useOnePass,
+      Map<Object, Integer> localCache) {
     DataType dataType = carbonColumn.getDataType();
     if (DataTypes.isArrayType(dataType)) {
       List<CarbonDimension> listOfChildDimensions =
@@ -151,8 +158,8 @@ public class FieldEncoderFactory {
           new ArrayDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
       for (CarbonDimension dimension : listOfChildDimensions) {
         arrayDataType.addChildren(
-            createComplexType(dimension, carbonColumn.getColName(), cache, carbonTableIdentifier,
-                client, useOnePass, storePath, localCache));
+            createComplexType(dimension, carbonColumn.getColName(), cache, absoluteTableIdentifier,
+                client, useOnePass, localCache));
       }
       return arrayDataType;
     } else if (DataTypes.isStructType(dataType)) {
@@ -163,16 +170,16 @@ public class FieldEncoderFactory {
           new StructDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
       for (CarbonDimension dimension : dimensions) {
         structDataType.addChildren(
-            createComplexType(dimension, carbonColumn.getColName(), cache, carbonTableIdentifier,
-                client, useOnePass, storePath, localCache));
+            createComplexType(dimension, carbonColumn.getColName(), cache, absoluteTableIdentifier,
+                client, useOnePass, localCache));
       }
       return structDataType;
     } else if (DataTypes.isMapType(dataType)) {
       throw new UnsupportedOperationException("Complex type Map is not supported yet");
     } else {
       return new PrimitiveDataType(carbonColumn.getColName(), parentName,
-          carbonColumn.getColumnId(), (CarbonDimension) carbonColumn, cache, carbonTableIdentifier,
-          client, useOnePass, storePath, localCache);
+          carbonColumn.getColumnId(), (CarbonDimension) carbonColumn, cache,
+          absoluteTableIdentifier, client, useOnePass, localCache);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
index 79c6d61..16c4a22 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
@@ -85,8 +85,7 @@ public class RowConverterImpl implements RowConverter {
   @Override
   public void initialize() throws IOException {
     CacheProvider cacheProvider = CacheProvider.getInstance();
-    cache = cacheProvider.createCache(CacheType.REVERSE_DICTIONARY,
-        configuration.getTableIdentifier().getStorePath());
+    cache = cacheProvider.createCache(CacheType.REVERSE_DICTIONARY);
     String nullFormat =
         configuration.getDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT)
             .toString();
@@ -102,10 +101,8 @@ public class RowConverterImpl implements RowConverter {
     for (int i = 0; i < fields.length; i++) {
       localCaches[i] = new ConcurrentHashMap<>();
       FieldConverter fieldConverter = FieldEncoderFactory.getInstance()
-          .createFieldEncoder(fields[i], cache,
-              configuration.getTableIdentifier().getCarbonTableIdentifier(), i, nullFormat, client,
-              configuration.getUseOnePass(), configuration.getTableIdentifier().getStorePath(),
-              localCaches[i], isEmptyBadRecord);
+          .createFieldEncoder(fields[i], cache, configuration.getTableIdentifier(), i, nullFormat,
+              client, configuration.getUseOnePass(), localCaches[i], isEmptyBadRecord);
       fieldConverterList.add(fieldConverter);
     }
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
@@ -210,10 +207,9 @@ public class RowConverterImpl implements RowConverter {
     for (int i = 0; i < fields.length; i++) {
       FieldConverter fieldConverter = null;
       try {
-        fieldConverter = FieldEncoderFactory.getInstance().createFieldEncoder(fields[i], cache,
-            configuration.getTableIdentifier().getCarbonTableIdentifier(), i, nullFormat, client,
-            configuration.getUseOnePass(), configuration.getTableIdentifier().getStorePath(),
-            localCaches[i], isEmptyBadRecord);
+        fieldConverter = FieldEncoderFactory.getInstance()
+            .createFieldEncoder(fields[i], cache, configuration.getTableIdentifier(), i, nullFormat,
+                client, configuration.getUseOnePass(), localCaches[i], isEmptyBadRecord);
       } catch (IOException e) {
         throw new RuntimeException(e);
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
index acd7fed..8c3fe56 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
@@ -43,7 +43,7 @@ public class CarbonLoadModel implements Serializable {
 
   private boolean aggLoadRequest;
 
-  private String storePath;
+  private String tablePath;
 
   private boolean isRetentionRequest;
 
@@ -354,7 +354,7 @@ public class CarbonLoadModel implements Serializable {
     copy.dateFormat = dateFormat;
     copy.defaultTimestampFormat = defaultTimestampFormat;
     copy.maxColumns = maxColumns;
-    copy.storePath = storePath;
+    copy.tablePath = tablePath;
     copy.useOnePass = useOnePass;
     copy.dictionaryServerHost = dictionaryServerHost;
     copy.dictionaryServerPort = dictionaryServerPort;
@@ -402,7 +402,7 @@ public class CarbonLoadModel implements Serializable {
     copy.dateFormat = dateFormat;
     copy.defaultTimestampFormat = defaultTimestampFormat;
     copy.maxColumns = maxColumns;
-    copy.storePath = storePath;
+    copy.tablePath = tablePath;
     copy.useOnePass = useOnePass;
     copy.dictionaryServerHost = dictionaryServerHost;
     copy.dictionaryServerPort = dictionaryServerPort;
@@ -452,7 +452,7 @@ public class CarbonLoadModel implements Serializable {
     copyObj.dateFormat = dateFormat;
     copyObj.defaultTimestampFormat = defaultTimestampFormat;
     copyObj.maxColumns = maxColumns;
-    copyObj.storePath = storePath;
+    copyObj.tablePath = tablePath;
     copyObj.useOnePass = useOnePass;
     copyObj.dictionaryServerHost = dictionaryServerHost;
     copyObj.dictionaryServerPort = dictionaryServerPort;
@@ -480,17 +480,17 @@ public class CarbonLoadModel implements Serializable {
   }
 
   /**
-   * @param storePath The storePath to set.
+   * @param tablePath The tablePath to set.
    */
-  public void setStorePath(String storePath) {
-    this.storePath = storePath;
+  public void setTablePath(String tablePath) {
+    this.tablePath = tablePath;
   }
 
   /**
    * @return Returns the factStoreLocation.
    */
-  public String getStorePath() {
-    return storePath;
+  public String getTablePath() {
+    return tablePath;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/main/java/org/apache/carbondata/processing/merger/AbstractResultProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/AbstractResultProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/AbstractResultProcessor.java
index aa77fb6..6da50a9 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/AbstractResultProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/AbstractResultProcessor.java
@@ -47,7 +47,7 @@ public abstract class AbstractResultProcessor {
     CarbonDataFileAttributes carbonDataFileAttributes;
     if (compactionType == CompactionType.IUD_UPDDEL_DELTA_COMPACTION) {
       int taskNo = CarbonUpdateUtil.getLatestTaskIdForSegment(loadModel.getSegmentId(),
-          CarbonStorePath.getCarbonTablePath(loadModel.getStorePath(),
+          CarbonStorePath.getCarbonTablePath(loadModel.getTablePath(),
               carbonTable.getCarbonTableIdentifier()));
       // Increase the Task Index as in IUD_UPDDEL_DELTA_COMPACTION the new file will
       // be written in same segment. So the TaskNo should be incremented by 1 from max val.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index c2ee1bc..c1df349 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -171,9 +171,7 @@ public final class CarbonDataMergerUtil {
     AbsoluteTableIdentifier absoluteTableIdentifier =
         carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
+    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
 
     SegmentUpdateStatusManager segmentUpdateStatusManager =
         new SegmentUpdateStatusManager(absoluteTableIdentifier);
@@ -298,8 +296,7 @@ public final class CarbonDataMergerUtil {
             + carbonLoadModel.getTableName() + " for table status updation ");
 
         CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
-                absoluteTableIdentifier.getCarbonTableIdentifier());
+            .getCarbonTablePath(absoluteTableIdentifier);
 
         String statusFilePath = carbonTablePath.getTableStatusFilePath();
 
@@ -388,7 +385,7 @@ public final class CarbonDataMergerUtil {
   public static List<LoadMetadataDetails> identifySegmentsToBeMerged(
       CarbonLoadModel carbonLoadModel, long compactionSize,
       List<LoadMetadataDetails> segments, CompactionType compactionType) {
-    String storeLocation = carbonLoadModel.getStorePath();
+    String tablePath = carbonLoadModel.getTablePath();
     List<LoadMetadataDetails> sortedSegments = new ArrayList<LoadMetadataDetails>(segments);
 
     sortSegments(sortedSegments);
@@ -413,7 +410,7 @@ public final class CarbonDataMergerUtil {
     if (compactionType.equals(CompactionType.MAJOR_COMPACTION)) {
 
       listOfSegmentsToBeMerged = identifySegmentsToBeMergedBasedOnSize(compactionSize,
-          listOfSegmentsLoadedInSameDateInterval, carbonLoadModel, storeLocation);
+          listOfSegmentsLoadedInSameDateInterval, carbonLoadModel, tablePath);
     } else {
 
       listOfSegmentsToBeMerged =
@@ -580,7 +577,7 @@ public final class CarbonDataMergerUtil {
    */
   private static List<LoadMetadataDetails> identifySegmentsToBeMergedBasedOnSize(
       long compactionSize, List<LoadMetadataDetails> listOfSegmentsAfterPreserve,
-      CarbonLoadModel carbonLoadModel, String storeLocation) {
+      CarbonLoadModel carbonLoadModel, String tablePath) {
 
     List<LoadMetadataDetails> segmentsToBeMerged =
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -603,7 +600,7 @@ public final class CarbonDataMergerUtil {
       String segId = segment.getLoadName();
       // variable to store one  segment size across partition.
       long sizeOfOneSegmentAcrossPartition =
-          getSizeOfSegment(storeLocation, tableIdentifier, segId);
+          getSizeOfSegment(tablePath, tableIdentifier, segId);
 
       // if size of a segment is greater than the Major compaction size. then ignore it.
       if (sizeOfOneSegmentAcrossPartition > (compactionSize * 1024 * 1024)) {
@@ -646,9 +643,9 @@ public final class CarbonDataMergerUtil {
    * @param segId segment id
    * @return the data size of the segment
    */
-  private static long getSizeOfSegment(String storePath,
+  private static long getSizeOfSegment(String tablePath,
       CarbonTableIdentifier tableIdentifier, String segId) {
-    String loadPath = getStoreLocation(storePath, tableIdentifier, segId);
+    String loadPath = getStoreLocation(tablePath, tableIdentifier, segId);
     CarbonFile segmentFolder =
         FileFactory.getCarbonFile(loadPath, FileFactory.getFileType(loadPath));
     return getSizeOfFactFileInLoad(segmentFolder);
@@ -657,15 +654,15 @@ public final class CarbonDataMergerUtil {
   /**
    * This method will get the store location for the given path, segemnt id and partition id
    *
-   * @param storePath the store path of the segment
+   * @param tablePath
    * @param carbonTableIdentifier identifier of catbon table that the segment belong to
    * @param segmentId segment id
    * @return the store location of the segment
    */
-  private static String getStoreLocation(String storePath,
+  private static String getStoreLocation(String tablePath,
       CarbonTableIdentifier carbonTableIdentifier, String segmentId) {
     CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier);
+        CarbonStorePath.getCarbonTablePath(tablePath, carbonTableIdentifier);
     return carbonTablePath.getCarbonDataDirectoryPath("0", segmentId);
   }
 
@@ -1001,9 +998,7 @@ public final class CarbonDataMergerUtil {
     CarbonFile[] updateDeltaFiles = null;
     Set<String> uniqueBlocks = new HashSet<String>();
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
+    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
 
     String segmentPath = carbonTablePath.getCarbonDataDirectoryPath("0", seg);
     CarbonFile segDir =
@@ -1255,8 +1250,7 @@ public final class CarbonDataMergerUtil {
     AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier();
 
     CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
-                    absoluteTableIdentifier.getCarbonTableIdentifier());
+            .getCarbonTablePath(absoluteTableIdentifier);
 
     String tableStatusPath = carbonTablePath.getTableStatusFilePath();
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
index b6ac19d..4f9458c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -27,6 +27,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.TableSpec;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
@@ -34,7 +35,6 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -306,7 +306,7 @@ public class CarbonFactDataHandlerModel {
     }
     carbonFactDataHandlerModel.setMeasureDataType(measureDataTypes);
     String carbonDataDirectoryPath = CarbonDataProcessorUtil
-        .checkAndCreateCarbonStoreLocation(loadModel.getStorePath(), loadModel.getDatabaseName(),
+        .checkAndCreateCarbonStoreLocation(loadModel.getTablePath(), loadModel.getDatabaseName(),
             tableName, loadModel.getPartitionId(), loadModel.getSegmentId());
     carbonFactDataHandlerModel.setCarbonDataDirectoryPath(carbonDataDirectoryPath);
     List<CarbonDimension> dimensionByTableName = carbonTable.getDimensionByTableName(tableName);
@@ -331,17 +331,13 @@ public class CarbonFactDataHandlerModel {
    * @return data directory path
    */
   private static String getCarbonDataFolderLocation(CarbonDataLoadConfiguration configuration) {
-    String carbonStorePath =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS);
-    CarbonTableIdentifier tableIdentifier =
-        configuration.getTableIdentifier().getCarbonTableIdentifier();
-    CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(
-        tableIdentifier.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + tableIdentifier
-            .getTableName());
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTable.getCarbonTableIdentifier());
-    return carbonTablePath.getCarbonDataDirectoryPath(configuration.getPartitionId(),
-        configuration.getSegmentId() + "");
+    AbsoluteTableIdentifier absoluteTableIdentifier = configuration.getTableIdentifier();
+    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
+    String carbonDataDirectoryPath = carbonTablePath
+        .getCarbonDataDirectoryPath(configuration.getPartitionId(),
+            configuration.getSegmentId() + "");
+    CarbonUtil.checkAndCreateFolder(carbonDataDirectoryPath);
+    return carbonDataDirectoryPath;
   }
 
   public int[] getColCardinality() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index 5b6998a..29a979d 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -89,7 +89,7 @@ public final class CarbonLoaderUtil {
   public static void deleteSegment(CarbonLoadModel loadModel, int currentLoad) {
     CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(loadModel.getStorePath(), carbonTable.getCarbonTableIdentifier());
+        .getCarbonTablePath(loadModel.getTablePath(), carbonTable.getCarbonTableIdentifier());
 
     for (int i = 0; i < carbonTable.getPartitionCount(); i++) {
       String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(i + "", currentLoad + "");
@@ -109,7 +109,7 @@ public final class CarbonLoaderUtil {
     CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema()
         .getCarbonTable();
     CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(
-        loadModel.getStorePath(), carbonTable.getCarbonTableIdentifier());
+        loadModel.getTablePath(), carbonTable.getCarbonTableIdentifier());
 
     int fileCount = 0;
     int partitionCount = carbonTable.getPartitionCount();
@@ -145,7 +145,7 @@ public final class CarbonLoaderUtil {
     String metaDataLocation = carbonTable.getMetaDataFilepath();
     final LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(metaDataLocation);
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(loadModel.getStorePath(), carbonTable.getCarbonTableIdentifier());
+        .getCarbonTablePath(loadModel.getTablePath(), carbonTable.getCarbonTableIdentifier());
 
     //delete folder which metadata no exist in tablestatus
     for (int i = 0; i < carbonTable.getPartitionCount(); i++) {
@@ -262,9 +262,7 @@ public final class CarbonLoaderUtil {
         loadModel.getCarbonDataLoadSchema().getCarbonTable().getMetaDataFilepath();
     AbsoluteTableIdentifier absoluteTableIdentifier =
         loadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
+    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     String tableStatusPath = carbonTablePath.getTableStatusFilePath();
     SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
@@ -382,10 +380,10 @@ public final class CarbonLoaderUtil {
     loadMetadataDetails.setLoadStartTime(loadStartTime);
   }
 
-  public static void writeLoadMetadata(String storeLocation, String dbName, String tableName,
+  public static void writeLoadMetadata(AbsoluteTableIdentifier absoluteTableIdentifier,
       List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
     CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(storeLocation, dbName, tableName);
+        CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     String dataLoadLocation = carbonTablePath.getTableStatusFilePath();
 
     DataOutputStream dataOutputStream;
@@ -427,20 +425,19 @@ public final class CarbonLoaderUtil {
     return date;
   }
 
-  public static Dictionary getDictionary(DictionaryColumnUniqueIdentifier columnIdentifier,
-      String carbonStorePath) throws IOException {
+  public static Dictionary getDictionary(DictionaryColumnUniqueIdentifier columnIdentifier)
+      throws IOException {
     Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictCache =
-        CacheProvider.getInstance().createCache(CacheType.REVERSE_DICTIONARY, carbonStorePath);
+        CacheProvider.getInstance().createCache(CacheType.REVERSE_DICTIONARY);
     return dictCache.get(columnIdentifier);
   }
 
-  public static Dictionary getDictionary(CarbonTableIdentifier tableIdentifier,
-      ColumnIdentifier columnIdentifier, String carbonStorePath, DataType dataType)
+  public static Dictionary getDictionary(AbsoluteTableIdentifier absoluteTableIdentifier,
+      ColumnIdentifier columnIdentifier, DataType dataType)
       throws IOException {
     return getDictionary(
-        new DictionaryColumnUniqueIdentifier(tableIdentifier, columnIdentifier, dataType,
-            CarbonStorePath.getCarbonTablePath(carbonStorePath, tableIdentifier)),
-        carbonStorePath);
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier, dataType,
+            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java b/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
index ffdabce..09e7b47 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
@@ -24,7 +24,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
@@ -43,17 +43,14 @@ public final class DeleteLoadFolders {
   /**
    * returns segment path
    *
-   * @param dbName
-   * @param tableName
-   * @param storeLocation
+   * @param absoluteTableIdentifier
    * @param partitionId
    * @param oneLoad
    * @return
    */
-  private static String getSegmentPath(String dbName, String tableName, String storeLocation,
+  private static String getSegmentPath(AbsoluteTableIdentifier absoluteTableIdentifier,
       int partitionId, LoadMetadataDetails oneLoad) {
-    CarbonTablePath carbon = new CarbonStorePath(storeLocation).getCarbonTablePath(
-        new CarbonTableIdentifier(dbName, tableName, ""));
+    CarbonTablePath carbon = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     String segmentId = oneLoad.getLoadName();
     return carbon.getCarbonDataDirectoryPath("" + partitionId, segmentId);
   }
@@ -125,15 +122,15 @@ public final class DeleteLoadFolders {
     return false;
   }
 
-  public static boolean deleteLoadFoldersFromFileSystem(String dbName, String tableName,
-      String storeLocation, boolean isForceDelete, LoadMetadataDetails[] details) {
-
+  public static boolean deleteLoadFoldersFromFileSystem(
+      AbsoluteTableIdentifier absoluteTableIdentifier, boolean isForceDelete,
+      LoadMetadataDetails[] details) {
     boolean isDeleted = false;
 
     if (details != null && details.length != 0) {
       for (LoadMetadataDetails oneLoad : details) {
         if (checkIfLoadCanBeDeleted(oneLoad, isForceDelete)) {
-          String path = getSegmentPath(dbName, tableName, storeLocation, 0, oneLoad);
+          String path = getSegmentPath(absoluteTableIdentifier, 0, oneLoad);
           boolean deletionStatus = physicalFactAndMeasureMetadataDeletion(path);
           if (deletionStatus) {
             isDeleted = true;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
index d15b45c..485e718 100644
--- a/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
+++ b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
@@ -63,7 +63,7 @@ public class BlockIndexStoreTest extends TestCase {
     CarbonProperties.getInstance().
         addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
     CacheProvider cacheProvider = CacheProvider.getInstance();
-    cache = (BlockIndexStore) cacheProvider.createCache(CacheType.EXECUTOR_BTREE, "");
+    cache = (BlockIndexStore) cacheProvider.createCache(CacheType.EXECUTOR_BTREE);
   }
 
   @AfterClass public void tearDown() {
@@ -258,9 +258,7 @@ public class BlockIndexStoreTest extends TestCase {
   }
 
   private static File getPartFile() {
-    String path = StoreCreator.getAbsoluteTableIdentifier().getStorePath() + "/" + StoreCreator
-        .getAbsoluteTableIdentifier().getCarbonTableIdentifier().getDatabaseName() + "/"
-        + StoreCreator.getAbsoluteTableIdentifier().getCarbonTableIdentifier().getTableName()
+    String path = StoreCreator.getAbsoluteTableIdentifier().getTablePath()
         + "/Fact/Part0/Segment_0";
     File file = new File(path);
     File[] files = file.listFiles();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java b/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
index ddc8657..ba77f29 100644
--- a/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
+++ b/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
@@ -18,6 +18,7 @@ package org.apache.carbondata.lcm.locks;
 
 import java.io.File;
 
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.locks.LocalFileLock;
 import org.apache.carbondata.core.locks.LockUsage;
@@ -51,13 +52,15 @@ public class LocalFileLockTest {
 
   @Test public void testingLocalFileLockingByAcquiring2Locks() {
 
-	CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("databaseName", "tableName", "tableId");
+    AbsoluteTableIdentifier absoluteTableIdentifier = AbsoluteTableIdentifier
+        .from(CarbonProperties.getInstance().getProperty("carbon.storelocation"), "databaseName",
+            "tableName");
     LocalFileLock localLock1 =
-        new LocalFileLock(carbonTableIdentifier,
+        new LocalFileLock(absoluteTableIdentifier,
             LockUsage.METADATA_LOCK);
     Assert.assertTrue(localLock1.lock());
     LocalFileLock localLock2 =
-        new LocalFileLock(carbonTableIdentifier,
+        new LocalFileLock(absoluteTableIdentifier,
             LockUsage.METADATA_LOCK);
     Assert.assertTrue(!localLock2.lock());
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/test/java/org/apache/carbondata/lcm/locks/ZooKeeperLockingTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/lcm/locks/ZooKeeperLockingTest.java b/processing/src/test/java/org/apache/carbondata/lcm/locks/ZooKeeperLockingTest.java
index 757f2e1..e0b6d66 100644
--- a/processing/src/test/java/org/apache/carbondata/lcm/locks/ZooKeeperLockingTest.java
+++ b/processing/src/test/java/org/apache/carbondata/lcm/locks/ZooKeeperLockingTest.java
@@ -17,6 +17,9 @@
 package org.apache.carbondata.lcm.locks;
 
 import mockit.NonStrictExpectations;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.locks.LockUsage;
@@ -95,7 +98,9 @@ public class ZooKeeperLockingTest {
 
     ZookeeperInit zki = ZookeeperInit.getInstance("127.0.0.1:" + freePort);
 
-    CarbonTableIdentifier tableIdentifier = new CarbonTableIdentifier("dbName", "tableName", "tableId");
+    AbsoluteTableIdentifier tableIdentifier = AbsoluteTableIdentifier
+        .from(CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION),
+            "dbName", "tableName");
     ZooKeeperLocking zkl =
         new ZooKeeperLocking(tableIdentifier,
             LockUsage.METADATA_LOCK);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
index 0f919ab..58cc019 100644
--- a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
+++ b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
@@ -99,14 +99,15 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 public class StoreCreator {
 
   private static AbsoluteTableIdentifier absoluteTableIdentifier;
-
+  private static String storePath = "";
   static {
     try {
-      String storePath = new File("target/store").getCanonicalPath();
+      storePath = new File("target/store").getCanonicalPath();
       String dbName = "testdb";
       String tableName = "testtable";
       absoluteTableIdentifier =
-          new AbsoluteTableIdentifier(storePath, new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
+          new AbsoluteTableIdentifier(storePath + "/testdb/testtable",
+              new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
     } catch (IOException ex) {
 
     }
@@ -122,10 +123,10 @@ public class StoreCreator {
   public static void createCarbonStore() {
     try {
       String factFilePath = new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
-      File storeDir = new File(absoluteTableIdentifier.getStorePath());
+      File storeDir = new File(storePath);
       CarbonUtil.deleteFoldersAndFiles(storeDir);
       CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS,
-          absoluteTableIdentifier.getStorePath());
+          storePath);
 
       CarbonTable table = createTable();
       writeDictionary(factFilePath, table);
@@ -137,7 +138,7 @@ public class StoreCreator {
       loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
       loadModel.setFactFilePath(factFilePath);
       loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
-      loadModel.setStorePath(absoluteTableIdentifier.getStorePath());
+      loadModel.setTablePath(absoluteTableIdentifier.getTablePath());
       loadModel.setDateFormat(null);
       loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
           CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
@@ -165,7 +166,7 @@ public class StoreCreator {
       loadModel.setFactTimeStamp(System.currentTimeMillis());
       loadModel.setMaxColumns("10");
 
-      loadData(loadModel, absoluteTableIdentifier.getStorePath());
+      loadData(loadModel, storePath);
 
     } catch (Exception e) {
       e.printStackTrace();
@@ -174,7 +175,7 @@ public class StoreCreator {
 
   private static CarbonTable createTable() throws IOException {
     TableInfo tableInfo = new TableInfo();
-    tableInfo.setStorePath(absoluteTableIdentifier.getStorePath());
+    tableInfo.setTablePath(absoluteTableIdentifier.getTablePath());
     tableInfo.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
     TableSchema tableSchema = new TableSchema();
     tableSchema.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
@@ -263,7 +264,7 @@ public class StoreCreator {
     tableInfo.setFactTable(tableSchema);
 
     CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier());
     String schemaFilePath = carbonTablePath.getSchemaFilePath();
     String schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath);
@@ -314,21 +315,24 @@ public class StoreCreator {
     }
 
     Cache dictCache = CacheProvider.getInstance()
-        .createCache(CacheType.REVERSE_DICTIONARY, absoluteTableIdentifier.getStorePath());
+        .createCache(CacheType.REVERSE_DICTIONARY);
     for (int i = 0; i < set.length; i++) {
-      ColumnIdentifier columnIdentifier = new ColumnIdentifier(dims.get(i).getColumnId(), null, null);
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier = new DictionaryColumnUniqueIdentifier(table.getCarbonTableIdentifier(), columnIdentifier, columnIdentifier.getDataType(),
-          CarbonStorePath.getCarbonTablePath(table.getStorePath(), table.getCarbonTableIdentifier()));
+      ColumnIdentifier columnIdentifier =
+          new ColumnIdentifier(dims.get(i).getColumnId(), null, null);
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
+          new DictionaryColumnUniqueIdentifier(table.getAbsoluteTableIdentifier(), columnIdentifier,
+              columnIdentifier.getDataType(), CarbonStorePath
+              .getCarbonTablePath(table.getAbsoluteTableIdentifier().getTablePath(),
+                  table.getCarbonTableIdentifier()));
       CarbonDictionaryWriter writer =
-          new CarbonDictionaryWriterImpl(absoluteTableIdentifier.getStorePath(),
-              absoluteTableIdentifier.getCarbonTableIdentifier(), dictionaryColumnUniqueIdentifier);
+          new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
       for (String value : set[i]) {
         writer.write(value);
       }
       writer.close();
       writer.commit();
       Dictionary dict = (Dictionary) dictCache.get(
-          new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier.getCarbonTableIdentifier(),
+          new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
         		  columnIdentifier, dims.get(i).getDataType(),
               CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)));
       CarbonDictionarySortInfoPreparator preparator =
@@ -337,9 +341,7 @@ public class StoreCreator {
       CarbonDictionarySortInfo dictionarySortInfo =
           preparator.getDictionarySortInfo(newDistinctValues, dict, dims.get(i).getDataType());
       CarbonDictionarySortIndexWriter carbonDictionaryWriter =
-          new CarbonDictionarySortIndexWriterImpl(
-              absoluteTableIdentifier.getCarbonTableIdentifier(), dictionaryColumnUniqueIdentifier,
-              absoluteTableIdentifier.getStorePath());
+          new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
       try {
         carbonDictionaryWriter.writeSortIndex(dictionarySortInfo.getSortIndex());
         carbonDictionaryWriter.writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted());
@@ -405,7 +407,7 @@ public class StoreCreator {
         format.createRecordReader(blockDetails, hadoopAttemptContext);
 
     CSVRecordReaderIterator readerIterator = new CSVRecordReaderIterator(recordReader, blockDetails, hadoopAttemptContext);
-    String[] storeLocationArray = new String[] {storeLocation};
+    String[] storeLocationArray = new String[] {storeLocation + "/" + databaseName + "/" + tableName};
     new DataLoadExecutor().execute(loadModel,
         storeLocationArray,
         new CarbonIterator[]{readerIterator});
@@ -483,7 +485,7 @@ public class StoreCreator {
   }
 
   public static String readCurrentTime() {
-    SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP);
+    SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS);
     String date = null;
 
     date = sdf.format(new Date());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala
index 80936d1..6ee3296 100644
--- a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala
+++ b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala
@@ -38,7 +38,7 @@ class CarbonStreamingQueryListener(spark: SparkSession) extends StreamingQueryLi
       LOGGER.info("Carbon streaming query started: " + event.id)
       val sink = qry.sink.asInstanceOf[CarbonAppendableStreamSink]
       val carbonTable = sink.carbonTable
-      val lock = CarbonLockFactory.getCarbonLockObj(carbonTable.getCarbonTableIdentifier,
+      val lock = CarbonLockFactory.getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier,
         LockUsage.STREAMING_LOCK)
       if (lock.lockWithRetries()) {
         LOGGER.info("Acquired the lock for stream table: " + carbonTable.getDatabaseName + "." +


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

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index 2671aad..c7db436 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -70,7 +70,6 @@ case class CarbonDictionaryDecoder(
 
   override def doExecute(): RDD[InternalRow] = {
     attachTree(this, "execute") {
-      val storePath = CarbonEnv.getInstance(sparkSession).storePath
       val absoluteTableIdentifiers = relations.map { relation =>
         val carbonTable = relation.carbonRelation.carbonRelation.metaData.carbonTable
         (carbonTable.getFactTableName, carbonTable.getAbsoluteTableIdentifier)
@@ -81,7 +80,7 @@ case class CarbonDictionaryDecoder(
         child.execute().mapPartitions { iter =>
           val cacheProvider: CacheProvider = CacheProvider.getInstance
           val forwardDictionaryCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
-            cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, storePath)
+            cacheProvider.createCache(CacheType.FORWARD_DICTIONARY)
           val dicts: Seq[Dictionary] = getDictionary(absoluteTableIdentifiers,
             forwardDictionaryCache)
           val dictIndex = dicts.zipWithIndex.filter(x => x._1 != null).map(x => x._2)
@@ -124,7 +123,6 @@ case class CarbonDictionaryDecoder(
 
   override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
 
-    val storePath = CarbonEnv.getInstance(sparkSession).storePath
     val absoluteTableIdentifiers = relations.map { relation =>
       val carbonTable = relation.carbonRelation.carbonRelation.metaData.carbonTable
       (carbonTable.getFactTableName, carbonTable.getAbsoluteTableIdentifier)
@@ -133,9 +131,9 @@ case class CarbonDictionaryDecoder(
     if (CarbonDictionaryDecoder.isRequiredToDecode(getDictionaryColumnIds)) {
       val cacheProvider: CacheProvider = CacheProvider.getInstance
       val forwardDictionaryCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
-        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, storePath)
+        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY)
       val dicts: Seq[ForwardDictionaryWrapper] = getDictionaryWrapper(absoluteTableIdentifiers,
-        forwardDictionaryCache, storePath)
+        forwardDictionaryCache)
 
       val exprs = child.output.map { exp =>
         ExpressionCanonicalizer.execute(BindReferences.bindReference(exp, child.output))
@@ -252,7 +250,7 @@ case class CarbonDictionaryDecoder(
       if (f._2 != null) {
         try {
           cache.get(new DictionaryColumnUniqueIdentifier(
-            atiMap(f._1).getCarbonTableIdentifier,
+            atiMap(f._1),
             f._2, f._3.getDataType,
             CarbonStorePath.getCarbonTablePath(atiMap(f._1))))
         } catch {
@@ -266,33 +264,31 @@ case class CarbonDictionaryDecoder(
   }
 
   private def getDictionaryWrapper(atiMap: Map[String, AbsoluteTableIdentifier],
-      cache: Cache[DictionaryColumnUniqueIdentifier, Dictionary], storePath: String) = {
+      cache: Cache[DictionaryColumnUniqueIdentifier, Dictionary]) = {
     val allDictIdentifiers = new ArrayBuffer[DictionaryColumnUniqueIdentifier]()
     val dicts: Seq[ForwardDictionaryWrapper] = getDictionaryColumnIds.map {
       case (tableName, columnIdentifier, carbonDimension) =>
         if (columnIdentifier != null) {
           try {
-            val (newCarbonTableIdentifier, newColumnIdentifier) =
+            val (newAbsoluteTableIdentifier, newColumnIdentifier) =
               if (null != carbonDimension.getColumnSchema.getParentColumnTableRelations &&
                   !carbonDimension
                     .getColumnSchema.getParentColumnTableRelations.isEmpty) {
-                (QueryUtil.getTableIdentifierForColumn(carbonDimension),
+                (QueryUtil.getTableIdentifierForColumn(carbonDimension, atiMap(tableName)),
                   new ColumnIdentifier(carbonDimension.getColumnSchema
                     .getParentColumnTableRelations.get(0).getColumnId,
                     carbonDimension.getColumnProperties,
                     carbonDimension.getDataType))
               } else {
-                (atiMap(tableName).getCarbonTableIdentifier, columnIdentifier)
+                (atiMap(tableName), columnIdentifier)
               }
             val dictionaryColumnUniqueIdentifier = new DictionaryColumnUniqueIdentifier(
-              newCarbonTableIdentifier,
+              newAbsoluteTableIdentifier,
               newColumnIdentifier, carbonDimension.getDataType,
               CarbonStorePath
-                .getCarbonTablePath(atiMap(tableName).getStorePath, newCarbonTableIdentifier))
+                .getCarbonTablePath(newAbsoluteTableIdentifier))
             allDictIdentifiers += dictionaryColumnUniqueIdentifier
-            new ForwardDictionaryWrapper(
-              storePath,
-              dictionaryColumnUniqueIdentifier)
+            new ForwardDictionaryWrapper(dictionaryColumnUniqueIdentifier)
           } catch {
             case _: Throwable => null
           }
@@ -300,7 +296,7 @@ case class CarbonDictionaryDecoder(
           null
         }
     }
-    val dictionaryLoader = new DictionaryLoader(storePath, allDictIdentifiers.toList)
+    val dictionaryLoader = new DictionaryLoader(allDictIdentifiers.toList)
     dicts.foreach { dict =>
       if (dict != null) {
         dict.setDictionaryLoader(dictionaryLoader)
@@ -467,7 +463,6 @@ class CarbonDecoderRDD(
     aliasMap: CarbonAliasDecoderRelation,
     prev: RDD[InternalRow],
     output: Seq[Attribute],
-    storePath: String,
     serializedTableInfo: Array[Byte])
   extends CarbonRDDWithTableInfo[InternalRow](prev, serializedTableInfo) {
 
@@ -516,7 +511,7 @@ class CarbonDecoderRDD(
 
     val cacheProvider: CacheProvider = CacheProvider.getInstance
     val forwardDictionaryCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
-      cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, storePath)
+      cacheProvider.createCache(CacheType.FORWARD_DICTIONARY)
     val dicts: Seq[Dictionary] = getDictionary(absoluteTableIdentifiers,
       forwardDictionaryCache)
     val dictIndex = dicts.zipWithIndex.filter(x => x._1 != null).map(x => x._2)
@@ -559,7 +554,7 @@ class CarbonDecoderRDD(
       if (f._2 != null) {
         try {
           cache.get(new DictionaryColumnUniqueIdentifier(
-            atiMap(f._1).getCarbonTableIdentifier,
+            atiMap(f._1),
             f._2, f._3.getDataType,
             CarbonStorePath.getCarbonTablePath(atiMap(f._1))))
         } catch {
@@ -578,10 +573,9 @@ class CarbonDecoderRDD(
 /**
  * It is a wrapper around Dictionary, it is a work around to keep the dictionary serializable in
  * case of codegen
- * @param storePath
+ * @param dictIdentifier Dictionary column unique identifier
  */
 class ForwardDictionaryWrapper(
-    val storePath: String,
     dictIdentifier: DictionaryColumnUniqueIdentifier) extends Serializable {
 
   var dictionary: Dictionary = null
@@ -610,8 +604,8 @@ class ForwardDictionaryWrapper(
 /**
  * It is Dictionary Loader class to load all dictionaries at a time instead of one by one.
  */
-class DictionaryLoader(storePath: String,
-    allDictIdentifiers: List[DictionaryColumnUniqueIdentifier]) extends Serializable {
+class DictionaryLoader(allDictIdentifiers: List[DictionaryColumnUniqueIdentifier])
+  extends Serializable {
 
   var isDictionaryLoaded = false
 
@@ -621,7 +615,7 @@ class DictionaryLoader(storePath: String,
     if (!isDictionaryLoaded) {
       val cacheProvider: CacheProvider = CacheProvider.getInstance
       val forwardDictionaryCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
-        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, storePath)
+        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY)
       allDicts = forwardDictionaryCache.getAll(allDictIdentifiers.asJava)
       isDictionaryLoaded = true
       val dictionaryTaskCleaner = TaskContext.get

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
index de01c8d..fba590e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
@@ -222,10 +222,21 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
 
     // check "tablePath" option
     val tablePathOption = parameters.get("tablePath")
+    val dbName: String = parameters.getOrElse("dbName",
+      CarbonCommonConstants.DATABASE_DEFAULT_NAME).toLowerCase
+    val tableOption: Option[String] = parameters.get("tableName")
+    if (tableOption.isEmpty) {
+      throw new CarbonStreamException("Table creation failed. Table name is not specified")
+    }
+    val tableName = tableOption.get.toLowerCase()
+    if (tableName.contains(" ")) {
+      throw new CarbonStreamException("Table creation failed. Table name cannot contain blank " +
+                                      "space")
+    }
     if (tablePathOption.isDefined) {
       val sparkSession = sqlContext.sparkSession
       val identifier: AbsoluteTableIdentifier =
-        AbsoluteTableIdentifier.fromTablePath(tablePathOption.get)
+        AbsoluteTableIdentifier.from(tablePathOption.get, dbName, tableName)
       val carbonTable =
         CarbonEnv.getInstance(sparkSession).carbonMetastore.
           createCarbonRelation(parameters, identifier, sparkSession).tableMeta.carbonTable
@@ -303,18 +314,20 @@ object CarbonSource {
     val tableName: String = properties.getOrElse("tableName", "").toLowerCase
     val model = createTableInfoFromParams(properties, dataSchema, dbName, tableName)
     val tableInfo: TableInfo = TableNewProcessor(model)
-    val tablePath = CarbonEnv.getInstance(sparkSession).storePath + "/" + dbName + "/" + tableName
+    val dbLocation = GetDB.getDatabaseLocation(dbName, sparkSession,
+      CarbonEnv.getInstance(sparkSession).storePath)
+    val tablePath = dbLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName
     val schemaEvolutionEntry = new SchemaEvolutionEntry
     schemaEvolutionEntry.setTimeStamp(tableInfo.getLastUpdatedTime)
     tableInfo.getFactTable.getSchemaEvalution.
       getSchemaEvolutionEntryList.add(schemaEvolutionEntry)
     val map = if (metaStore.isReadFromHiveMetaStore) {
-      val tableIdentifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
+      val tableIdentifier = AbsoluteTableIdentifier.from(tablePath, dbName, tableName)
       val carbonTablePath = CarbonStorePath.getCarbonTablePath(tableIdentifier)
       val schemaMetadataPath =
         CarbonTablePath.getFolderContainingFile(carbonTablePath.getSchemaFilePath)
       tableInfo.setMetaDataFilepath(schemaMetadataPath)
-      tableInfo.setStorePath(tableIdentifier.getStorePath)
+      tableInfo.setTablePath(tableIdentifier.getTablePath)
       CarbonUtil.convertToMultiStringMap(tableInfo)
     } else {
       metaStore.saveToDisk(tableInfo, tablePath)
@@ -322,6 +335,7 @@ object CarbonSource {
     }
     properties.foreach(e => map.put(e._1, e._2))
     map.put("tablePath", tablePath)
+    map.put("dbname", dbName)
     map.asScala.toMap
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala
index f5c6cba..197b23b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala
@@ -39,9 +39,11 @@ case class CarbonCreateTableCommand(
   override def processSchema(sparkSession: SparkSession): Seq[Row] = {
     val storePath = CarbonEnv.getInstance(sparkSession).storePath
     CarbonEnv.getInstance(sparkSession).carbonMetastore.
-      checkSchemasModifiedTimeAndReloadTables(storePath)
+      checkSchemasModifiedTimeAndReloadTables()
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
     cm.databaseName = GetDB.getDatabaseName(cm.databaseNameOp, sparkSession)
+    val dbLocation = GetDB.getDatabaseLocation(cm.databaseName, sparkSession, storePath)
+    val tablePath = dbLocation + CarbonCommonConstants.FILE_SEPARATOR + cm.tableName
     val tbName = cm.tableName
     val dbName = cm.databaseName
     LOGGER.audit(s"Creating Table with Database name [$dbName] and Table name [$tbName]")
@@ -70,11 +72,10 @@ case class CarbonCreateTableCommand(
         sys.error(s"Table [$tbName] already exists under database [$dbName]")
       }
     } else {
-      val tableIdentifier = AbsoluteTableIdentifier.from(storePath, dbName, tbName)
+      val tableIdentifier = AbsoluteTableIdentifier.from(tablePath, dbName, tbName)
       // Add Database to catalog and persist
       val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      val tablePath = tableIdentifier.getTablePath
-      val carbonSchemaString = catalog.generateTableSchemaString(tableInfo, tablePath)
+      val carbonSchemaString = catalog.generateTableSchemaString(tableInfo, tableIdentifier)
       if (createDSTable) {
         try {
           val fields = new Array[Field](cm.dimCols.size + cm.msrCols.size)
@@ -89,10 +90,9 @@ case class CarbonCreateTableCommand(
             s""""$tablePath"$carbonSchemaString) """)
         } catch {
           case e: Exception =>
-            val identifier: TableIdentifier = TableIdentifier(tbName, Some(dbName))
             // call the drop table to delete the created table.
             CarbonEnv.getInstance(sparkSession).carbonMetastore
-              .dropTable(tablePath, identifier)(sparkSession)
+              .dropTable(tableIdentifier)(sparkSession)
 
             LOGGER.audit(s"Table creation with Database name [$dbName] " +
                          s"and Table name [$tbName] failed")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
index 1bf17b3..0343393 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
@@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.hive.CarbonRelation
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
@@ -53,14 +54,16 @@ case class CarbonDropTableCommand(
     val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.DROP_TABLE_LOCK)
     val carbonEnv = CarbonEnv.getInstance(sparkSession)
     val catalog = carbonEnv.carbonMetastore
-    val tableIdentifier =
-      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath,
-        dbName.toLowerCase, tableName.toLowerCase)
-    catalog.checkSchemasModifiedTimeAndReloadTables(tableIdentifier.getStorePath)
+    val databaseLocation = GetDB.getDatabaseLocation(dbName, sparkSession,
+      CarbonEnv.getInstance(sparkSession).storePath)
+    val tablePath = databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName.toLowerCase
+    val absoluteTableIdentifier = AbsoluteTableIdentifier
+      .from(tablePath, dbName.toLowerCase, tableName.toLowerCase)
+    catalog.checkSchemasModifiedTimeAndReloadTables()
     val carbonLocks: scala.collection.mutable.ListBuffer[ICarbonLock] = ListBuffer()
     try {
       locksToBeAcquired foreach {
-        lock => carbonLocks += CarbonLockUtil.getLockObject(carbonTableIdentifier, lock)
+        lock => carbonLocks += CarbonLockUtil.getLockObject(absoluteTableIdentifier, lock)
       }
       LOGGER.audit(s"Deleting table [$tableName] under database [$dbName]")
       val carbonTable: Option[CarbonTable] =
@@ -98,7 +101,7 @@ case class CarbonDropTableCommand(
           sparkSession)
       OperationListenerBus.getInstance.fireEvent(dropTablePreEvent, operationContext)
       CarbonEnv.getInstance(sparkSession).carbonMetastore
-        .dropTable(tableIdentifier.getTablePath, identifier)(sparkSession)
+        .dropTable(absoluteTableIdentifier)(sparkSession)
 
       // fires the event after dropping main table
       val dropTablePostEvent: DropTablePostEvent =
@@ -127,8 +130,10 @@ case class CarbonDropTableCommand(
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     // delete the table folder
     val dbName = GetDB.getDatabaseName(databaseNameOp, sparkSession)
-    val tableIdentifier =
-      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath, dbName, tableName)
+    val databaseLocation = GetDB.getDatabaseLocation(dbName, sparkSession,
+      CarbonEnv.getInstance(sparkSession).storePath)
+    val tablePath = databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName.toLowerCase
+    val tableIdentifier = AbsoluteTableIdentifier.from(tablePath, dbName, tableName)
     val metadataFilePath =
       CarbonStorePath.getCarbonTablePath(tableIdentifier).getMetadataDirectoryPath
     val fileType = FileFactory.getFileType(metadataFilePath)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
index dc3b1ae..66f2756 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
@@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
 import org.apache.spark.sql.hive.CarbonRelation
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
@@ -63,14 +64,16 @@ case class CarbonDropDataMapCommand(
     val locksToBeAcquired = List(LockUsage.METADATA_LOCK)
     val carbonEnv = CarbonEnv.getInstance(sparkSession)
     val catalog = carbonEnv.carbonMetastore
+    val databaseLocation = GetDB.getDatabaseLocation(dbName, sparkSession,
+      CarbonEnv.getInstance(sparkSession).storePath)
+    val tablePath = databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName.toLowerCase
     val tableIdentifier =
-      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath,
-        dbName.toLowerCase, tableName.toLowerCase)
-    catalog.checkSchemasModifiedTimeAndReloadTables(tableIdentifier.getStorePath)
+      AbsoluteTableIdentifier.from(tablePath, dbName.toLowerCase, tableName.toLowerCase)
+    catalog.checkSchemasModifiedTimeAndReloadTables()
     val carbonLocks: scala.collection.mutable.ListBuffer[ICarbonLock] = ListBuffer()
     try {
       locksToBeAcquired foreach {
-        lock => carbonLocks += CarbonLockUtil.getLockObject(carbonTableIdentifier, lock)
+        lock => carbonLocks += CarbonLockUtil.getLockObject(tableIdentifier, lock)
       }
       LOGGER.audit(s"Deleting datamap [$dataMapName] under table [$tableName]")
       val carbonTable: Option[CarbonTable] =
@@ -140,8 +143,10 @@ case class CarbonDropDataMapCommand(
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     // delete the table folder
     val dbName = GetDB.getDatabaseName(databaseNameOp, sparkSession)
-    val tableIdentifier =
-      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath, dbName, tableName)
+    val databaseLocation = GetDB.getDatabaseLocation(dbName, sparkSession,
+      CarbonEnv.getInstance(sparkSession).storePath)
+    val tablePath = databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + tableName.toLowerCase
+    val tableIdentifier = AbsoluteTableIdentifier.from(tablePath, dbName, tableName)
     DataMapStoreManager.getInstance().clearDataMap(tableIdentifier, dataMapName)
     Seq.empty
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
index f87e734..947cea1 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
@@ -71,7 +71,7 @@ case class AlterTableCompactionCommand(
     carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
     carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
     carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
-    carbonLoadModel.setStorePath(relation.tableMeta.carbonTable.getStorePath)
+    carbonLoadModel.setTablePath(relation.tableMeta.carbonTable.getTablePath)
 
     var storeLocation = CarbonProperties.getInstance
       .getProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
@@ -131,7 +131,7 @@ case class AlterTableCompactionCommand(
       // Just launch job to merge index and return
       CommonUtil.mergeIndexFiles(sqlContext.sparkContext,
         carbonLoadModel.getLoadMetadataDetails.asScala.map(_.getLoadName),
-        carbonLoadModel.getStorePath,
+        carbonLoadModel.getTablePath,
         carbonTable)
       return
     }
@@ -172,7 +172,7 @@ case class AlterTableCompactionCommand(
     } else {
       // normal flow of compaction
       val lock = CarbonLockFactory
-        .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+        .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier,
           LockUsage.COMPACTION_LOCK
         )
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
index 1b16b88..8b0dab7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
@@ -36,10 +36,14 @@ case class CleanFilesCommand(
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
     if (forceTableClean) {
+      val dbName = GetDB.getDatabaseName(databaseNameOp, sparkSession)
+      val databaseLocation = GetDB.getDatabaseLocation(dbName, sparkSession,
+        CarbonEnv.getInstance(sparkSession).storePath)
+      // TODO: TAABLEPATH
       CarbonStore.cleanFiles(
-        GetDB.getDatabaseName(databaseNameOp, sparkSession),
+        dbName,
         tableName,
-        CarbonEnv.getInstance(sparkSession).storePath,
+        databaseLocation,
         null,
         forceTableClean)
     } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
index 3853b5f..777c169 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
@@ -192,7 +192,7 @@ case class LoadTableCommand(
       } finally {
         // Once the data load is successful delete the unwanted partition files
         try {
-          val partitionLocation = table.getStorePath + "/partition/" +
+          val partitionLocation = relation.tableMeta.storePath + "/partition/" +
                                   table.getDatabaseName + "/" +
                                   table.getFactTableName + "/"
           val fileType = FileFactory.getFileType(partitionLocation)
@@ -231,7 +231,7 @@ case class LoadTableCommand(
     val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
       .getCarbonTableIdentifier
     val carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(carbonLoadModel.getStorePath, carbonTableIdentifier)
+      .getCarbonTablePath(carbonLoadModel.getTablePath, carbonTableIdentifier)
     val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
     val dimensions = carbonTable.getDimensionByTableName(
       carbonTable.getFactTableName).asScala.toArray
@@ -245,7 +245,7 @@ case class LoadTableCommand(
         dimensions,
         carbonLoadModel,
         sparkSession.sqlContext,
-        carbonLoadModel.getStorePath,
+        carbonLoadModel.getTablePath,
         dictFolderPath)
     }
     if (!StringUtils.isEmpty(carbonLoadModel.getAllDictPath)) {
@@ -253,7 +253,7 @@ case class LoadTableCommand(
       GlobalDictionaryUtil
         .generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
           carbonLoadModel,
-          carbonLoadModel.getStorePath,
+          carbonLoadModel.getTablePath,
           carbonTableIdentifier,
           dictFolderPath,
           dimensions,
@@ -289,7 +289,7 @@ case class LoadTableCommand(
     }
     CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
       carbonLoadModel,
-      carbonLoadModel.getStorePath,
+      carbonLoadModel.getTablePath,
       columnar,
       partitionStatus,
       server,
@@ -332,11 +332,11 @@ case class LoadTableCommand(
     GlobalDictionaryUtil.generateGlobalDictionary(
       sparkSession.sqlContext,
       carbonLoadModel,
-      carbonLoadModel.getStorePath,
+      carbonLoadModel.getTablePath,
       dictionaryDataFrame)
     CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
       carbonLoadModel,
-      carbonLoadModel.getStorePath,
+      carbonLoadModel.getTablePath,
       columnar,
       partitionStatus,
       None,
@@ -351,8 +351,7 @@ case class LoadTableCommand(
       model: DictionaryLoadModel,
       noDictDimension: Array[CarbonDimension]): Unit = {
     val sparkSession = sqlContext.sparkSession
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(model.hdfsLocation,
-      model.table)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(model.table)
 
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     // read TableInfo
@@ -374,11 +373,12 @@ case class LoadTableCommand(
       tableInfo, entry, carbonTablePath.getPath)(sparkSession)
 
     // update the schema modified time
-    metastore.updateAndTouchSchemasUpdatedTime(model.hdfsLocation)
+    metastore.updateAndTouchSchemasUpdatedTime()
 
+    val identifier = model.table.getCarbonTableIdentifier
     // update CarbonDataLoadSchema
-    val carbonTable = metastore.lookupRelation(Option(model.table.getDatabaseName),
-      model.table.getTableName)(sqlContext.sparkSession).asInstanceOf[CarbonRelation].tableMeta
+    val carbonTable = metastore.lookupRelation(Option(identifier.getDatabaseName),
+      identifier.getTableName)(sqlContext.sparkSession).asInstanceOf[CarbonRelation].tableMeta
       .carbonTable
     carbonLoadModel.setCarbonDataLoadSchema(new CarbonDataLoadSchema(carbonTable))
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
index 0c39dd4..a52008a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
@@ -76,14 +76,10 @@ object DeleteExecution {
       .lookupRelation(DeleteExecution.getTableIdentifier(identifier))(sparkSession).
       asInstanceOf[CarbonRelation]
 
-    val storeLocation = relation.tableMeta.storePath
-    val absoluteTableIdentifier: AbsoluteTableIdentifier = new
-        AbsoluteTableIdentifier(storeLocation,
-          relation.tableMeta.carbonTableIdentifier)
-    val tablePath = CarbonStorePath.getCarbonTablePath(
-      storeLocation,
-      absoluteTableIdentifier.getCarbonTableIdentifier)
-    val factPath = tablePath.getFactDir
+    val absoluteTableIdentifier = relation.tableMeta.carbonTable.getAbsoluteTableIdentifier
+    val carbonTablePath = CarbonStorePath
+      .getCarbonTablePath(absoluteTableIdentifier)
+    val factPath = carbonTablePath.getFactDir
 
     val carbonTable = relation.tableMeta.carbonTable
     var deleteStatus = true

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
index 764deb7..a898822 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
@@ -60,7 +60,7 @@ private[sql] case class ProjectForDeleteCommand(
     OperationListenerBus.getInstance.fireEvent(deleteFromTablePreEvent, operationContext)
 
     val metadataLock = CarbonLockFactory
-      .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+      .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier,
         LockUsage.METADATA_LOCK)
     var lockStatus = false
     try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
index e48693b..549c58f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
@@ -67,7 +67,7 @@ private[sql] case class ProjectForUpdateCommand(
     OperationListenerBus.getInstance.fireEvent(updateTablePreEvent, operationContext)
 
     val metadataLock = CarbonLockFactory
-      .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+      .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier,
         LockUsage.METADATA_LOCK)
     var lockStatus = false
     // get the current time stamp which should be same for delete and update.
@@ -83,9 +83,7 @@ private[sql] case class ProjectForUpdateCommand(
       else {
         throw new Exception("Table is locked for updation. Please try after some time")
       }
-      val tablePath = CarbonStorePath.getCarbonTablePath(
-        carbonTable.getStorePath,
-        carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier)
+      val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
       // Get RDD.
 
       dataSet = if (isPersistEnabled) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
index 9b16060..acd9bd3 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
@@ -66,8 +66,8 @@ case class AlterTableDropCarbonPartitionCommand(
     val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
       .asInstanceOf[CarbonRelation]
     val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
-    val storePath = relation.tableMeta.storePath
-    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables(storePath)
+    val tablePath = relation.tableMeta.tablePath
+    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables()
     if (relation == null) {
       sys.error(s"Table $dbName.$tableName does not exist")
     }
@@ -101,14 +101,14 @@ case class AlterTableDropCarbonPartitionCommand(
         sys.error(s"Dropping range interval partition isn't support yet!")
     }
     partitionInfo.dropPartition(partitionIndex)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(tablePath, carbonTableIdentifier)
     val schemaFilePath = carbonTablePath.getSchemaFilePath
     // read TableInfo
     val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
 
     val schemaConverter = new ThriftWrapperSchemaConverterImpl()
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
-      dbName, tableName, storePath)
+      dbName, tableName, tablePath)
     val tableSchema = wrapperTableInfo.getFactTable
     tableSchema.setPartitionInfo(partitionInfo)
     wrapperTableInfo.setFactTable(tableSchema)
@@ -118,10 +118,10 @@ case class AlterTableDropCarbonPartitionCommand(
     thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
       .setTime_stamp(System.currentTimeMillis)
     carbonMetaStore.updateMetadataByThriftTable(schemaFilePath, thriftTable,
-      dbName, tableName, storePath)
+      dbName, tableName, tablePath)
     CarbonUtil.writeThriftTableToSchemaFile(schemaFilePath, thriftTable)
     // update the schema modified time
-    carbonMetaStore.updateAndTouchSchemasUpdatedTime(storePath)
+    carbonMetaStore.updateAndTouchSchemasUpdatedTime()
     // sparkSession.catalog.refreshTable(tableName)
     Seq.empty
   }
@@ -152,7 +152,7 @@ case class AlterTableDropCarbonPartitionCommand(
       carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
       carbonLoadModel.setTableName(carbonTableIdentifier.getTableName)
       carbonLoadModel.setDatabaseName(carbonTableIdentifier.getDatabaseName)
-      carbonLoadModel.setStorePath(relation.tableMeta.storePath)
+      carbonLoadModel.setTablePath(relation.tableMeta.tablePath)
       val loadStartTime = CarbonUpdateUtil.readCurrentTime
       carbonLoadModel.setFactTimeStamp(loadStartTime)
       alterTableDropPartition(
@@ -224,7 +224,7 @@ case class AlterTableDropCarbonPartitionCommand(
       for (thread <- threadArray) {
         thread.join()
       }
-      val identifier = AbsoluteTableIdentifier.from(carbonLoadModel.getStorePath,
+      val identifier = AbsoluteTableIdentifier.from(carbonLoadModel.getTablePath,
         carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
       val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(identifier)
       refresher.refreshSegments(validSegments.asJava)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
index c3a918c..0973226 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
@@ -70,11 +70,11 @@ case class AlterTableSplitCarbonPartitionCommand(
     val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
       .asInstanceOf[CarbonRelation]
     val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
-    val storePath = relation.tableMeta.storePath
+    val tablePath = relation.tableMeta.tablePath
     if (relation == null) {
       sys.error(s"Table $dbName.$tableName does not exist")
     }
-    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables(storePath)
+    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables()
     if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)) {
       LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
       sys.error(s"Alter table failed. table not found: $dbName.$tableName")
@@ -95,13 +95,13 @@ case class AlterTableSplitCarbonPartitionCommand(
 
     updatePartitionInfo(partitionInfo, partitionIds)
 
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(tablePath, carbonTableIdentifier)
     val schemaFilePath = carbonTablePath.getSchemaFilePath
     // read TableInfo
     val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
     val schemaConverter = new ThriftWrapperSchemaConverterImpl()
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
-      dbName, tableName, storePath)
+      dbName, tableName, tablePath)
     val tableSchema = wrapperTableInfo.getFactTable
     tableSchema.setPartitionInfo(partitionInfo)
     wrapperTableInfo.setFactTable(tableSchema)
@@ -109,10 +109,10 @@ case class AlterTableSplitCarbonPartitionCommand(
     val thriftTable =
       schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
     carbonMetaStore.updateMetadataByThriftTable(schemaFilePath, thriftTable,
-      dbName, tableName, storePath)
+      dbName, tableName, tablePath)
     CarbonUtil.writeThriftTableToSchemaFile(schemaFilePath, thriftTable)
     // update the schema modified time
-    carbonMetaStore.updateAndTouchSchemasUpdatedTime(storePath)
+    carbonMetaStore.updateAndTouchSchemasUpdatedTime()
     sparkSession.sessionState.catalog.refreshTable(TableIdentifier(tableName, Option(dbName)))
     Seq.empty
   }
@@ -153,14 +153,14 @@ case class AlterTableSplitCarbonPartitionCommand(
       val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
       val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
         .asInstanceOf[CarbonRelation]
-      val storePath = relation.tableMeta.storePath
+      val tablePath = relation.tableMeta.tablePath
       val table = relation.tableMeta.carbonTable
       val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
       val dataLoadSchema = new CarbonDataLoadSchema(table)
       carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
       carbonLoadModel.setTableName(carbonTableIdentifier.getTableName)
       carbonLoadModel.setDatabaseName(carbonTableIdentifier.getDatabaseName)
-      carbonLoadModel.setStorePath(storePath)
+      carbonLoadModel.setTablePath(tablePath)
       val loadStartTime = CarbonUpdateUtil.readCurrentTime
       carbonLoadModel.setFactTimeStamp(loadStartTime)
       alterTableSplitPartition(
@@ -232,7 +232,7 @@ case class AlterTableSplitCarbonPartitionCommand(
       threadArray.foreach {
         thread => thread.join()
       }
-      val identifier = AbsoluteTableIdentifier.from(carbonLoadModel.getStorePath,
+      val identifier = AbsoluteTableIdentifier.from(carbonLoadModel.getTablePath,
         carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
       val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(identifier)
       refresher.refreshSegments(validSegments.asJava)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
index d693061..3193310 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -332,15 +332,15 @@ object PreAggregateUtil {
       locks = acquireLock(dbName, tableName, locksToBeAcquired, carbonTable)
       // get the latest carbon table and check for column existence
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
+      val carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
       val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
       val wrapperTableInfo = schemaConverter
         .fromExternalToWrapperTableInfo(thriftTableInfo,
           dbName,
           tableName,
-          carbonTable.getStorePath)
+          carbonTable.getTablePath)
       numberOfCurrentChild = wrapperTableInfo.getDataMapSchemaList.size
       if (wrapperTableInfo.getDataMapSchemaList.asScala.
         exists(f => f.getDataMapName.equalsIgnoreCase(childSchema.getDataMapName))) {
@@ -399,7 +399,7 @@ object PreAggregateUtil {
     val acquiredLocks = ListBuffer[ICarbonLock]()
     try {
       locksToBeAcquired.foreach { lock =>
-        acquiredLocks += CarbonLockUtil.getLockObject(table.getCarbonTableIdentifier, lock)
+        acquiredLocks += CarbonLockUtil.getLockObject(table.getAbsoluteTableIdentifier, lock)
       }
       acquiredLocks.toList
     } catch {
@@ -439,13 +439,12 @@ object PreAggregateUtil {
       .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation].tableMeta
       .carbonTable
     carbonTable.getTableLastUpdatedTime
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-      carbonTable.getCarbonTableIdentifier)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
     val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
     if (thriftTable.dataMapSchemas.size > numberOfChildSchema) {
       metastore
-        .revertTableSchemaForPreAggCreationFailure(carbonTable.getCarbonTableIdentifier,
-          thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+        .revertTableSchemaForPreAggCreationFailure(carbonTable.getAbsoluteTableIdentifier,
+          thriftTable)(sparkSession)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
index 7cc43d2..2132131 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
@@ -65,26 +65,25 @@ private[sql] case class CarbonAlterTableAddColumnCommand(
       OperationListenerBus.getInstance().fireEvent(alterTableAddColumnListener)
       // get the latest carbon table and check for column existence
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
+      val carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
       val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
       val wrapperTableInfo = schemaConverter
         .fromExternalToWrapperTableInfo(thriftTableInfo,
           dbName,
           tableName,
-          carbonTable.getStorePath)
+          carbonTable.getTablePath)
       newCols = new AlterTableColumnSchemaGenerator(alterTableAddColumnsModel,
         dbName,
         wrapperTableInfo,
         carbonTablePath,
         carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath, sparkSession.sparkContext).process
+        sparkSession.sparkContext).process
       // generate dictionary files for the newly added columns
       new AlterTableAddColumnRDD(sparkSession.sparkContext,
         newCols,
-        carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath).collect()
+        carbonTable.getAbsoluteTableIdentifier).collect()
       timeStamp = System.currentTimeMillis
       val schemaEvolutionEntry = new org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
       schemaEvolutionEntry.setTimeStamp(timeStamp)
@@ -105,8 +104,7 @@ private[sql] case class CarbonAlterTableAddColumnCommand(
           LOGGER.info("Cleaning up the dictionary files as alter table add operation failed")
           new AlterTableDropColumnRDD(sparkSession.sparkContext,
             newCols,
-            carbonTable.getCarbonTableIdentifier,
-            carbonTable.getStorePath).collect()
+            carbonTable.getAbsoluteTableIdentifier).collect()
           AlterTableUtil.revertAddColumnChanges(dbName, tableName, timeStamp)(sparkSession)
         }
         sys.error(s"Alter table add operation failed: ${e.getMessage}")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
index 023e061..e44899e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
@@ -74,8 +74,8 @@ private[sql] case class CarbonAlterTableDataTypeChangeCommand(
         sys.error(s"Invalid Column: $columnName")
       }
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
+      val carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
       val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
       // maintain the added column for schema evolution history
       var addColumnSchema: ColumnSchema = null

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
index 0b737bf..dae2d7b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
@@ -110,8 +110,8 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
       OperationListenerBus.getInstance().fireEvent(alterTableDropColumnPreEvent, operationContext)
 
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
+      val carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
       val tableInfo: org.apache.carbondata.format.TableInfo =
         metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
       // maintain the deleted columns for schema evolution history
@@ -138,8 +138,7 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
       // delete dictionary files for dictionary column and clear dictionary cache from memory
       new AlterTableDropColumnRDD(sparkSession.sparkContext,
         dictionaryColumns,
-        carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath).collect()
+        carbonTable.getAbsoluteTableIdentifier).collect()
 
       // event will be fired before dropping the columns
       val alterTableDropColumnPostEvent: AlterTableDropColumnPostEvent =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index c2e5cf0..f1cce13 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -28,7 +28,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.CarbonStorePath
@@ -85,7 +85,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
         .asInstanceOf[CarbonRelation].tableMeta
       carbonTable = tableMeta.carbonTable
       // invalid data map for the old table, see CARBON-1690
-      val oldTableIdentifier = AbsoluteTableIdentifier.fromTablePath(tableMeta.tablePath)
+      val oldTableIdentifier = carbonTable.getAbsoluteTableIdentifier
       DataMapStoreManager.getInstance().clearDataMaps(oldTableIdentifier)
       // get the latest carbon table and check for column existence
       val carbonTablePath = CarbonStorePath.getCarbonTablePath(oldTableIdentifier)
@@ -106,6 +106,21 @@ private[sql] case class CarbonAlterTableRenameCommand(
       schemaEvolutionEntry.setTime_stamp(timeStamp)
       renameBadRecords(oldTableName, newTableName, oldDatabaseName)
       val fileType = FileFactory.getFileType(tableMetadataFile)
+      val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
+        newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
+      var newTablePath = CarbonUtil.getNewTablePath(carbonTablePath, newTableIdentifier)
+
+      metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
+      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
+        .runSqlHive(
+          s"ALTER TABLE $oldDatabaseName.$oldTableName RENAME TO $oldDatabaseName.$newTableName")
+      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
+        .runSqlHive(
+          s"ALTER TABLE $oldDatabaseName.$newTableName SET SERDEPROPERTIES" +
+          s"('tableName'='$newTableName', " +
+          s"'dbName'='$oldDatabaseName', 'tablePath'='$newTablePath')")
+      // changed the rename order to deal with situation when carbon table and hive table
+      // will point to the same tablePath
       if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
         val rename = FileFactory.getCarbonFile(carbonTablePath.getPath, fileType)
           .renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
@@ -115,23 +130,12 @@ private[sql] case class CarbonAlterTableRenameCommand(
           sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
         }
       }
-      val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
-        newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
-      val newTablePath = metastore.updateTableSchemaForAlter(newTableIdentifier,
+      newTablePath = metastore.updateTableSchemaForAlter(newTableIdentifier,
         carbonTable.getCarbonTableIdentifier,
         tableInfo,
         schemaEvolutionEntry,
         tableMeta.tablePath)(sparkSession)
 
-      metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
-      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
-        .runSqlHive(
-          s"ALTER TABLE $oldDatabaseName.$oldTableName RENAME TO $oldDatabaseName.$newTableName")
-      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
-        .runSqlHive(
-          s"ALTER TABLE $oldDatabaseName.$newTableName SET SERDEPROPERTIES" +
-          s"('tableName'='$newTableName', " +
-          s"'dbName'='$oldDatabaseName', 'tablePath'='$newTablePath')")
       val alterTableRenamePostEvent: AlterTableRenamePostEvent = AlterTableRenamePostEvent(
         carbonTable,
         alterTableRenameModel,
@@ -150,7 +154,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
           AlterTableUtil
             .revertRenameTableChanges(oldTableIdentifier,
               newTableName,
-              carbonTable.getStorePath,
+              carbonTable.getTablePath,
               carbonTable.getCarbonTableIdentifier.getTableId,
               timeStamp)(
               sparkSession)
@@ -167,7 +171,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
             locksToBeAcquired,
             oldDatabaseName,
             newTableName,
-            carbonTable.getStorePath)
+            carbonTable.getTablePath)
       }
     }
     Seq.empty

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
index c6ca950..c126b25 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
@@ -102,7 +102,6 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
       CarbonAliasDecoderRelation(),
       rdd,
       output,
-      CarbonEnv.getInstance(SparkSession.getActiveSession.get).storePath,
       table.carbonTable.getTableInfo.serialize())
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
index e39ba73..ef2e0a5 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -27,7 +27,6 @@ import org.apache.spark.sql.execution.command.partition.ShowCarbonPartitionsComm
 import org.apache.spark.sql.execution.command.schema._
 import org.apache.spark.sql.hive.execution.command.{CarbonDropDatabaseCommand, CarbonResetCommand, CarbonSetCommand}
 
-import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 /**
@@ -77,7 +76,6 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
       _, child: LogicalPlan, overwrite, _) =>
         ExecutedCommandExec(LoadTableByInsertCommand(relation, child, overwrite.enabled)) :: Nil
       case createDb@CreateDatabaseCommand(dbName, ifNotExists, _, _, _) =>
-        CarbonUtil.createDatabaseDirectory(dbName, CarbonEnv.getInstance(sparkSession).storePath)
         ExecutedCommandExec(createDb) :: Nil
       case drop@DropDatabaseCommand(dbName, ifExists, isCascade) =>
         ExecutedCommandExec(CarbonDropDatabaseCommand(drop)) :: Nil

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index 0343402..6d80a26 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -23,11 +23,10 @@ import java.util.concurrent.atomic.AtomicLong
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
-import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -39,12 +38,11 @@ import org.apache.carbondata.core.fileoperations.FileWriteOperation
 import org.apache.carbondata.core.metadata.{schema, AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.table
-import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier}
-import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.core.writer.ThriftWriter
 import org.apache.carbondata.events.{LookupRelationPostEvent, OperationContext, OperationListenerBus}
-import org.apache.carbondata.format
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
 import org.apache.carbondata.processing.merger.TableMeta
 import org.apache.carbondata.spark.util.CarbonSparkUtil
@@ -192,11 +190,11 @@ class CarbonFileMetastore extends CarbonMetaStore {
   private def readCarbonSchema(identifier: AbsoluteTableIdentifier): Option[TableMeta] = {
     val dbName = identifier.getCarbonTableIdentifier.getDatabaseName
     val tableName = identifier.getCarbonTableIdentifier.getTableName
-    val storePath = identifier.getStorePath
+    val tablePath = identifier.getTablePath
     val carbonTableIdentifier = new CarbonTableIdentifier(dbName.toLowerCase(),
       tableName.toLowerCase(), UUID.randomUUID().toString)
     val carbonTablePath =
-      CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
+      CarbonStorePath.getCarbonTablePath(tablePath, carbonTableIdentifier)
     val tableMetadataFile = carbonTablePath.getSchemaFilePath
     val fileType = FileFactory.getFileType(tableMetadataFile)
     if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
@@ -204,16 +202,16 @@ class CarbonFileMetastore extends CarbonMetaStore {
       val tableInfo: TableInfo = CarbonUtil.readSchemaFile(tableMetadataFile)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl
       val wrapperTableInfo = schemaConverter
-        .fromExternalToWrapperTableInfo(tableInfo, dbName, tableName, storePath)
+        .fromExternalToWrapperTableInfo(tableInfo, dbName, tableName, tablePath)
       val schemaFilePath = CarbonStorePath
-        .getCarbonTablePath(storePath, carbonTableIdentifier).getSchemaFilePath
-      wrapperTableInfo.setStorePath(storePath)
+        .getCarbonTablePath(tablePath, carbonTableIdentifier).getSchemaFilePath
+      wrapperTableInfo.setTablePath(tablePath)
       wrapperTableInfo
         .setMetaDataFilepath(CarbonTablePath.getFolderContainingFile(schemaFilePath))
       CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo)
       val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
       val tableMeta = new TableMeta(carbonTable.getCarbonTableIdentifier,
-        identifier.getStorePath,
+        identifier.getTablePath,
         identifier.getTablePath,
         carbonTable)
       metadata.tablesMeta += tableMeta
@@ -237,16 +235,19 @@ class CarbonFileMetastore extends CarbonMetaStore {
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
       schemaEvolutionEntry: SchemaEvolutionEntry,
       tablePath: String) (sparkSession: SparkSession): String = {
-    val absoluteTableIdentifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
+    val absoluteTableIdentifier = new AbsoluteTableIdentifier(tablePath, oldTableIdentifier)
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
     if (schemaEvolutionEntry != null) {
       thriftTableInfo.fact_table.schema_evolution.schema_evolution_history.add(schemaEvolutionEntry)
     }
+    val oldCarbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
+    val newAbsoluteTableIdentifier = new AbsoluteTableIdentifier(CarbonUtil
+      .getNewTablePath(oldCarbonTablePath, newTableIdentifier), newTableIdentifier)
     val wrapperTableInfo = schemaConverter
       .fromExternalToWrapperTableInfo(thriftTableInfo,
         newTableIdentifier.getDatabaseName,
         newTableIdentifier.getTableName,
-        absoluteTableIdentifier.getStorePath)
+        newAbsoluteTableIdentifier.getTablePath)
     val identifier =
       new CarbonTableIdentifier(newTableIdentifier.getDatabaseName,
         newTableIdentifier.getTableName,
@@ -254,10 +255,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
     val path = createSchemaThriftFile(wrapperTableInfo,
       thriftTableInfo,
       identifier)
-    addTableCache(wrapperTableInfo,
-      AbsoluteTableIdentifier.from(absoluteTableIdentifier.getStorePath,
-        newTableIdentifier.getDatabaseName,
-        newTableIdentifier.getTableName))
+    addTableCache(wrapperTableInfo, newAbsoluteTableIdentifier)
     path
   }
 
@@ -266,47 +264,44 @@ class CarbonFileMetastore extends CarbonMetaStore {
    *
    * @param carbonTableIdentifier
    * @param thriftTableInfo
-   * @param tablePath
    * @param sparkSession
    */
   def revertTableSchemaInAlterFailure(carbonTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
-      tablePath: String)(sparkSession: SparkSession): String = {
-    val tableIdentifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
+      absoluteTableIdentifier: AbsoluteTableIdentifier)(sparkSession: SparkSession): String = {
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
     val wrapperTableInfo = schemaConverter
       .fromExternalToWrapperTableInfo(thriftTableInfo,
         carbonTableIdentifier.getDatabaseName,
         carbonTableIdentifier.getTableName,
-        tableIdentifier.getStorePath)
+        absoluteTableIdentifier.getTablePath)
     val evolutionEntries = thriftTableInfo.fact_table.schema_evolution.schema_evolution_history
     evolutionEntries.remove(evolutionEntries.size() - 1)
-    wrapperTableInfo.setStorePath(tableIdentifier.getStorePath)
+    wrapperTableInfo.setTablePath(absoluteTableIdentifier.getTablePath)
     val path = createSchemaThriftFile(wrapperTableInfo,
       thriftTableInfo,
-      tableIdentifier.getCarbonTableIdentifier)
-    addTableCache(wrapperTableInfo, tableIdentifier)
+      absoluteTableIdentifier.getCarbonTableIdentifier)
+    addTableCache(wrapperTableInfo, absoluteTableIdentifier)
     path
   }
 
-  override def revertTableSchemaForPreAggCreationFailure(carbonTableIdentifier:
-  CarbonTableIdentifier,
-      thriftTableInfo: org.apache.carbondata.format.TableInfo,
-      tablePath: String)(sparkSession: SparkSession): String = {
-    val tableIdentifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
+  override def revertTableSchemaForPreAggCreationFailure(absoluteTableIdentifier:
+  AbsoluteTableIdentifier,
+      thriftTableInfo: org.apache.carbondata.format.TableInfo)
+    (sparkSession: SparkSession): String = {
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
     val wrapperTableInfo = schemaConverter
       .fromExternalToWrapperTableInfo(thriftTableInfo,
-        carbonTableIdentifier.getDatabaseName,
-        carbonTableIdentifier.getTableName,
-        tableIdentifier.getStorePath)
+        absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName,
+        absoluteTableIdentifier.getCarbonTableIdentifier.getTableName,
+        absoluteTableIdentifier.getTablePath)
     val childSchemaList = wrapperTableInfo.getDataMapSchemaList
     childSchemaList.remove(childSchemaList.size() - 1)
-    wrapperTableInfo.setStorePath(tableIdentifier.getStorePath)
+    wrapperTableInfo.setTablePath(absoluteTableIdentifier.getTablePath)
     val path = createSchemaThriftFile(wrapperTableInfo,
       thriftTableInfo,
-      tableIdentifier.getCarbonTableIdentifier)
-    addTableCache(wrapperTableInfo, tableIdentifier)
+      absoluteTableIdentifier.getCarbonTableIdentifier)
+    addTableCache(wrapperTableInfo, absoluteTableIdentifier)
     path
 
   }
@@ -323,8 +318,8 @@ class CarbonFileMetastore extends CarbonMetaStore {
     val tableName = tableInfo.getFactTable.getTableName
     val thriftTableInfo = schemaConverter
       .fromWrapperToExternalTableInfo(tableInfo, dbName, tableName)
-    val identifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
-    tableInfo.setStorePath(identifier.getStorePath)
+    val identifier = AbsoluteTableIdentifier.from(tablePath, dbName, tableName)
+    tableInfo.setTablePath(identifier.getTablePath)
     createSchemaThriftFile(tableInfo,
       thriftTableInfo,
       identifier.getCarbonTableIdentifier)
@@ -335,19 +330,18 @@ class CarbonFileMetastore extends CarbonMetaStore {
    * Generates schema string from TableInfo
    */
   override def generateTableSchemaString(tableInfo: schema.table.TableInfo,
-      tablePath: String): String = {
-    val tableIdentifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(tableIdentifier)
+      absoluteTableIdentifier: AbsoluteTableIdentifier): String = {
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
     val schemaMetadataPath =
       CarbonTablePath.getFolderContainingFile(carbonTablePath.getSchemaFilePath)
     tableInfo.setMetaDataFilepath(schemaMetadataPath)
-    tableInfo.setStorePath(tableIdentifier.getStorePath)
+    tableInfo.setTablePath(absoluteTableIdentifier.getTablePath)
     val schemaEvolutionEntry = new schema.SchemaEvolutionEntry
     schemaEvolutionEntry.setTimeStamp(tableInfo.getLastUpdatedTime)
     tableInfo.getFactTable.getSchemaEvalution.getSchemaEvolutionEntryList.add(schemaEvolutionEntry)
     removeTableFromMetadata(tableInfo.getDatabaseName, tableInfo.getFactTable.getTableName)
     CarbonMetadata.getInstance().loadTableMetadata(tableInfo)
-    addTableCache(tableInfo, tableIdentifier)
+    addTableCache(tableInfo, absoluteTableIdentifier)
     CarbonUtil.convertToMultiGsonStrings(tableInfo, " ", "", ",")
   }
 
@@ -362,7 +356,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
       thriftTableInfo: TableInfo,
       carbonTableIdentifier: CarbonTableIdentifier): String = {
     val carbonTablePath = CarbonStorePath.
-      getCarbonTablePath(tableInfo.getStorePath, carbonTableIdentifier)
+      getCarbonTablePath(tableInfo.getTablePath, carbonTableIdentifier)
     val schemaFilePath = carbonTablePath.getSchemaFilePath
     val schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath)
     tableInfo.setMetaDataFilepath(schemaMetadataPath)
@@ -374,7 +368,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
     thriftWriter.open(FileWriteOperation.OVERWRITE)
     thriftWriter.write(thriftTableInfo)
     thriftWriter.close()
-    updateSchemasUpdatedTime(touchSchemaFileSystemTime(tableInfo.getStorePath))
+    updateSchemasUpdatedTime(touchSchemaFileSystemTime())
     carbonTablePath.getPath
   }
 
@@ -384,7 +378,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
     CarbonMetadata.getInstance.removeTable(tableInfo.getTableUniqueName)
     removeTableFromMetadata(identifier.getDatabaseName, identifier.getTableName)
     CarbonMetadata.getInstance().loadTableMetadata(tableInfo)
-    val tableMeta = new TableMeta(identifier, absoluteTableIdentifier.getStorePath,
+    val tableMeta = new TableMeta(identifier, absoluteTableIdentifier.getTablePath,
       absoluteTableIdentifier.getTablePath,
       CarbonMetadata.getInstance().getCarbonTable(identifier.getTableUniqueName))
     metadata.tablesMeta += tableMeta
@@ -424,16 +418,16 @@ class CarbonFileMetastore extends CarbonMetaStore {
   }
 
   def updateMetadataByThriftTable(schemaFilePath: String,
-      tableInfo: TableInfo, dbName: String, tableName: String, storePath: String): Unit = {
+      tableInfo: TableInfo, dbName: String, tableName: String, tablePath: String): Unit = {
 
     tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
       .setTime_stamp(System.currentTimeMillis())
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
     val wrapperTableInfo = schemaConverter
-      .fromExternalToWrapperTableInfo(tableInfo, dbName, tableName, storePath)
+      .fromExternalToWrapperTableInfo(tableInfo, dbName, tableName, tablePath)
     wrapperTableInfo
       .setMetaDataFilepath(CarbonTablePath.getFolderContainingFile(schemaFilePath))
-    wrapperTableInfo.setStorePath(storePath)
+    wrapperTableInfo.setTablePath(tablePath)
     updateMetadataByWrapperTable(wrapperTableInfo)
   }
 
@@ -446,17 +440,17 @@ class CarbonFileMetastore extends CarbonMetaStore {
       FileFactory.isFileExist(tablePath, fileType)
     } catch {
       case e: Exception =>
-        false
+       false
     }
   }
 
 
-  def dropTable(tablePath: String, tableIdentifier: TableIdentifier)
+  def dropTable(absoluteTableIdentifier: AbsoluteTableIdentifier)
     (sparkSession: SparkSession) {
-    val dbName = tableIdentifier.database.get
-    val tableName = tableIdentifier.table
-    val identifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
-    val metadataFilePath = CarbonStorePath.getCarbonTablePath(identifier).getMetadataDirectoryPath
+    val dbName = absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName
+    val tableName = absoluteTableIdentifier.getCarbonTableIdentifier.getTableName
+    val metadataFilePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
+      .getMetadataDirectoryPath
     val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
     if (null != carbonTable) {
       // clear driver B-tree and dictionary cache
@@ -467,21 +461,28 @@ class CarbonFileMetastore extends CarbonMetaStore {
     if (FileFactory.isFileExist(metadataFilePath, fileType)) {
       // while drop we should refresh the schema modified time so that if any thing has changed
       // in the other beeline need to update.
-      checkSchemasModifiedTimeAndReloadTables(identifier.getStorePath)
+      checkSchemasModifiedTimeAndReloadTables()
 
       removeTableFromMetadata(dbName, tableName)
-
-      updateSchemasUpdatedTime(touchSchemaFileSystemTime(identifier.getStorePath))
+      updateSchemasUpdatedTime(touchSchemaFileSystemTime())
       CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sparkSession)
       // discard cached table info in cachedDataSourceTables
+      val tableIdentifier = TableIdentifier(tableName, Option(dbName))
       sparkSession.sessionState.catalog.refreshTable(tableIdentifier)
-      DataMapStoreManager.getInstance().clearDataMaps(identifier)
+      DataMapStoreManager.getInstance().clearDataMaps(absoluteTableIdentifier)
     }
   }
 
-  private def getTimestampFileAndType(basePath: String) = {
+  private def getTimestampFileAndType() = {
+    var basePath = CarbonProperties.getInstance()
+      .getProperty(CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER,
+        CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER_DEFAULT)
+    basePath = CarbonUtil.checkAndAppendFileSystemURIScheme(basePath)
     val timestampFile = basePath + "/" + CarbonCommonConstants.SCHEMAS_MODIFIED_TIME_FILE
     val timestampFileType = FileFactory.getFileType(timestampFile)
+    if (!FileFactory.isFileExist(basePath, timestampFileType)) {
+      FileFactory.mkdirs(basePath, timestampFileType)
+    }
     (timestampFile, timestampFileType)
   }
 
@@ -494,8 +495,8 @@ class CarbonFileMetastore extends CarbonMetaStore {
     tableModifiedTimeStore.put(CarbonCommonConstants.DATABASE_DEFAULT_NAME, timeStamp)
   }
 
-  def updateAndTouchSchemasUpdatedTime(basePath: String) {
-    updateSchemasUpdatedTime(touchSchemaFileSystemTime(basePath))
+  def updateAndTouchSchemasUpdatedTime() {
+    updateSchemasUpdatedTime(touchSchemaFileSystemTime())
   }
 
 
@@ -504,10 +505,10 @@ class CarbonFileMetastore extends CarbonMetaStore {
    *
    * @return
    */
-  private def touchSchemaFileSystemTime(basePath: String): Long = {
-    val (timestampFile, timestampFileType) = getTimestampFileAndType(basePath)
+  private def touchSchemaFileSystemTime(): Long = {
+    val (timestampFile, timestampFileType) = getTimestampFileAndType()
     if (!FileFactory.isFileExist(timestampFile, timestampFileType)) {
-      LOGGER.audit(s"Creating timestamp file for $basePath")
+      LOGGER.audit(s"Creating timestamp file for $timestampFile")
       FileFactory.createNewFile(timestampFile, timestampFileType)
     }
     FileFactory.getCarbonFile(timestampFile, timestampFileType)
@@ -518,9 +519,8 @@ class CarbonFileMetastore extends CarbonMetaStore {
       .getLastModifiedTime
   }
 
-  def checkSchemasModifiedTimeAndReloadTables(storePath: String) {
-    val (timestampFile, timestampFileType) =
-      getTimestampFileAndType(storePath)
+  def checkSchemasModifiedTimeAndReloadTables() {
+    val (timestampFile, timestampFileType) = getTimestampFileAndType()
     if (FileFactory.isFileExist(timestampFile, timestampFileType)) {
       if (!(FileFactory.getCarbonFile(timestampFile, timestampFileType).
         getLastModifiedTime ==

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
index a500f00..dedaf1c 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
@@ -18,7 +18,8 @@ package org.apache.spark.sql.hive
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
 
@@ -33,7 +34,7 @@ import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.format
 import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.processing.merger.TableMeta
-import org.apache.carbondata.spark.util.CarbonSparkUtil
+import org.apache.carbondata.spark.util.{CarbonSparkUtil, CommonUtil}
 
 /**
  * Metastore to store carbonschema in hive
@@ -56,7 +57,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
     if (info != null) {
       val table = CarbonTable.buildFromTableInfo(info)
       val meta = new TableMeta(table.getCarbonTableIdentifier,
-        absIdentifier.getStorePath, absIdentifier.getTablePath, table)
+        absIdentifier.getTablePath, absIdentifier.getTablePath, table)
       CarbonRelation(info.getDatabaseName, info.getFactTable.getTableName,
         CarbonSparkUtil.createSparkMeta(table), meta)
     } else {
@@ -70,25 +71,25 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
     tableExists(tableIdentifier)(sparkSession)
   }
 
-  override def dropTable(tablePath: String, tableIdentifier: TableIdentifier)
+  override def dropTable(absoluteTableIdentifier: AbsoluteTableIdentifier)
     (sparkSession: SparkSession): Unit = {
-    val dbName = tableIdentifier.database.get
-    val tableName = tableIdentifier.table
-    val identifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
+    val dbName = absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName
+    val tableName = absoluteTableIdentifier.getCarbonTableIdentifier.getTableName
     val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
     if (null != carbonTable) {
       // clear driver B-tree and dictionary cache
       ManageDictionaryAndBTree.clearBTreeAndDictionaryLRUCache(carbonTable)
     }
-    checkSchemasModifiedTimeAndReloadTables(identifier.getStorePath)
+    checkSchemasModifiedTimeAndReloadTables()
     removeTableFromMetadata(dbName, tableName)
     CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sparkSession)
     // discard cached table info in cachedDataSourceTables
+    val tableIdentifier = TableIdentifier(tableName, Option(dbName))
     sparkSession.sessionState.catalog.refreshTable(tableIdentifier)
-    DataMapStoreManager.getInstance().clearDataMaps(identifier)
+    DataMapStoreManager.getInstance().clearDataMaps(absoluteTableIdentifier)
   }
 
-  override def checkSchemasModifiedTimeAndReloadTables(storePath: String) {
+  override def checkSchemasModifiedTimeAndReloadTables() {
     // do nothing now
   }
 
@@ -125,14 +126,13 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
       tablePath: String)
     (sparkSession: SparkSession): String = {
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
-    val identifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
     if (schemaEvolutionEntry != null) {
       thriftTableInfo.fact_table.schema_evolution.schema_evolution_history.add(schemaEvolutionEntry)
     }
     updateHiveMetaStoreForAlter(newTableIdentifier,
       oldTableIdentifier,
       thriftTableInfo,
-      identifier.getStorePath,
+      tablePath,
       sparkSession,
       schemaConverter)
   }
@@ -142,19 +142,18 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
    *
    * @param newTableIdentifier
    * @param thriftTableInfo
-   * @param carbonStorePath
+   * @param carbonTablePath
    * @param sparkSession
    */
   override def updateTableSchemaForDataMap(newTableIdentifier: CarbonTableIdentifier,
       oldTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
-      carbonStorePath: String)(sparkSession: SparkSession): String = {
+      carbonTablePath: String)(sparkSession: SparkSession): String = {
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
-    val identifier = AbsoluteTableIdentifier.fromTablePath(carbonStorePath)
     updateHiveMetaStoreForDataMap(newTableIdentifier,
       oldTableIdentifier,
       thriftTableInfo,
-      identifier.getStorePath,
+      carbonTablePath,
       sparkSession,
       schemaConverter)
   }
@@ -165,13 +164,14 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
       carbonStorePath: String,
       sparkSession: SparkSession,
       schemaConverter: ThriftWrapperSchemaConverterImpl) = {
+    val tablePath = CarbonUtil.getNewTablePath(new Path(carbonStorePath), newTableIdentifier)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(tablePath, newTableIdentifier)
     val wrapperTableInfo = schemaConverter
       .fromExternalToWrapperTableInfo(thriftTableInfo,
         newTableIdentifier.getDatabaseName,
         newTableIdentifier.getTableName,
-        carbonStorePath)
-    wrapperTableInfo.setStorePath(carbonStorePath)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonStorePath, newTableIdentifier)
+        carbonTablePath.toString)
+    wrapperTableInfo.setTablePath(carbonStorePath)
     val schemaMetadataPath =
       CarbonTablePath.getFolderContainingFile(carbonTablePath.getSchemaFilePath)
     wrapperTableInfo.setMetaDataFilepath(schemaMetadataPath)
@@ -189,16 +189,17 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
   private def updateHiveMetaStoreForDataMap(newTableIdentifier: CarbonTableIdentifier,
       oldTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: format.TableInfo,
-      carbonStorePath: String,
+      tablePath: String,
       sparkSession: SparkSession,
       schemaConverter: ThriftWrapperSchemaConverterImpl) = {
+    val newTablePath = CarbonUtil.getNewTablePath(new Path(tablePath), newTableIdentifier)
     val wrapperTableInfo = schemaConverter
       .fromExternalToWrapperTableInfo(thriftTableInfo,
         newTableIdentifier.getDatabaseName,
         newTableIdentifier.getTableName,
-        carbonStorePath)
-    wrapperTableInfo.setStorePath(carbonStorePath)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonStorePath, newTableIdentifier)
+        newTablePath)
+    wrapperTableInfo.setTablePath(newTablePath)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(newTablePath, newTableIdentifier)
     val schemaMetadataPath =
       CarbonTablePath.getFolderContainingFile(carbonTablePath.getSchemaFilePath)
     wrapperTableInfo.setMetaDataFilepath(schemaMetadataPath)
@@ -207,7 +208,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
     sparkSession.catalog.refreshTable(TableIdentifier(tableName, Some(dbName)).quotedString)
     removeTableFromMetadata(dbName, tableName)
     CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo)
-    CarbonStorePath.getCarbonTablePath(carbonStorePath, newTableIdentifier).getPath
+    carbonTablePath.getPath
   }
 
   /**
@@ -219,32 +220,31 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
    */
   override def revertTableSchemaInAlterFailure(carbonTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: format.TableInfo,
-      tablePath: String)
+      identifier: AbsoluteTableIdentifier)
     (sparkSession: SparkSession): String = {
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
-    val identifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
     val evolutionEntries = thriftTableInfo.fact_table.schema_evolution.schema_evolution_history
     evolutionEntries.remove(evolutionEntries.size() - 1)
     updateHiveMetaStoreForAlter(carbonTableIdentifier,
       carbonTableIdentifier,
       thriftTableInfo,
-      identifier.getStorePath,
+      identifier.getTablePath,
       sparkSession,
       schemaConverter)
   }
 
-  override def revertTableSchemaForPreAggCreationFailure(carbonTableIdentifier:
-  CarbonTableIdentifier,
-      thriftTableInfo: org.apache.carbondata.format.TableInfo,
-      tablePath: String)(sparkSession: SparkSession): String = {
+  override def revertTableSchemaForPreAggCreationFailure(absoluteTableIdentifier:
+  AbsoluteTableIdentifier,
+      thriftTableInfo: org.apache.carbondata.format.TableInfo)
+    (sparkSession: SparkSession): String = {
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
-    val identifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
     val childSchemas = thriftTableInfo.dataMapSchemas
     childSchemas.remove(childSchemas.size())
+    val carbonTableIdentifier = absoluteTableIdentifier.getCarbonTableIdentifier
     updateHiveMetaStoreForAlter(carbonTableIdentifier,
       carbonTableIdentifier,
       thriftTableInfo,
-      identifier.getStorePath,
+      absoluteTableIdentifier.getTablePath,
       sparkSession,
       schemaConverter)
   }


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

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
index 348d9a7..9545f0f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
@@ -62,7 +62,6 @@ class C2DataMapFactory() extends DataMapFactory {
 }
 
 class DataMapWriterSuite extends QueryTest with BeforeAndAfterAll {
-
   def buildTestData(numRows: Int): DataFrame = {
     import sqlContext.implicits._
     sqlContext.sparkContext.parallelize(1 to numRows)
@@ -81,72 +80,74 @@ class DataMapWriterSuite extends QueryTest with BeforeAndAfterAll {
 
   test("test write datamap 2 pages") {
     // register datamap writer
-    DataMapStoreManager.getInstance().createAndRegisterDataMap(
-      AbsoluteTableIdentifier.from(storeLocation, "default", "carbon1"),
-      classOf[C2DataMapFactory].getName,
-      "test")
-
-    val df = buildTestData(33000)
-
-    // save dataframe to carbon file
-    df.write
-      .format("carbondata")
-      .option("tableName", "carbon1")
-      .mode(SaveMode.Overwrite)
-      .save()
-
-    assert(DataMapWriterSuite.callbackSeq.head.contains("block start"))
-    assert(DataMapWriterSuite.callbackSeq.last.contains("block end"))
-    assert(DataMapWriterSuite.callbackSeq.slice(1, DataMapWriterSuite.callbackSeq.length - 1) == Seq(
-      "blocklet start 0",
-      "add page data: blocklet 0, page 0",
-      "add page data: blocklet 0, page 1",
-      "blocklet end: 0"
-    ))
-    DataMapWriterSuite.callbackSeq = Seq()
+      DataMapStoreManager.getInstance().createAndRegisterDataMap(
+        AbsoluteTableIdentifier.from(storeLocation + "/carbon1", "default", "carbon1"),
+        classOf[C2DataMapFactory].getName,
+        "test")
+
+      val df = buildTestData(33000)
+
+      // save dataframe to carbon file
+      df.write
+        .format("carbondata")
+        .option("tableName", "carbon1")
+        .mode(SaveMode.Overwrite)
+        .save()
+
+      assert(DataMapWriterSuite.callbackSeq.head.contains("block start"))
+      assert(DataMapWriterSuite.callbackSeq.last.contains("block end"))
+      assert(
+        DataMapWriterSuite.callbackSeq.slice(1, DataMapWriterSuite.callbackSeq.length - 1) == Seq(
+          "blocklet start 0",
+          "add page data: blocklet 0, page 0",
+          "add page data: blocklet 0, page 1",
+          "blocklet end: 0"
+        ))
+      DataMapWriterSuite.callbackSeq = Seq()
   }
 
   test("test write datamap 2 blocklet") {
     // register datamap writer
-    DataMapStoreManager.getInstance().createAndRegisterDataMap(
-      AbsoluteTableIdentifier.from(storeLocation, "default", "carbon2"),
-      classOf[C2DataMapFactory].getName,
-      "test")
+      DataMapStoreManager.getInstance().createAndRegisterDataMap(
+        AbsoluteTableIdentifier.from(storeLocation + "/carbon2", "default", "carbon2"),
+        classOf[C2DataMapFactory].getName,
+        "test")
 
-    CarbonProperties.getInstance()
-      .addProperty("carbon.blockletgroup.size.in.mb", "1")
+      CarbonProperties.getInstance()
+        .addProperty("carbon.blockletgroup.size.in.mb", "1")
     CarbonProperties.getInstance()
       .addProperty("carbon.number.of.cores.while.loading",
           CarbonCommonConstants.NUM_CORES_DEFAULT_VAL)
 
-    val df = buildTestData(300000)
-
-    // save dataframe to carbon file
-    df.write
-      .format("carbondata")
-      .option("tableName", "carbon2")
-      .mode(SaveMode.Overwrite)
-      .save()
-
-    assert(DataMapWriterSuite.callbackSeq.head.contains("block start"))
-    assert(DataMapWriterSuite.callbackSeq.last.contains("block end"))
-    assert(DataMapWriterSuite.callbackSeq.slice(1, DataMapWriterSuite.callbackSeq.length - 1) == Seq(
-      "blocklet start 0",
-      "add page data: blocklet 0, page 0",
-      "add page data: blocklet 0, page 1",
-      "add page data: blocklet 0, page 2",
-      "add page data: blocklet 0, page 3",
-      "add page data: blocklet 0, page 4",
-      "add page data: blocklet 0, page 5",
-      "add page data: blocklet 0, page 6",
-      "add page data: blocklet 0, page 7",
-      "blocklet end: 0",
-      "blocklet start 1",
-      "add page data: blocklet 1, page 0",
-      "add page data: blocklet 1, page 1",
-      "blocklet end: 1"
-    ))
-    DataMapWriterSuite.callbackSeq = Seq()
+      val df = buildTestData(300000)
+
+      // save dataframe to carbon file
+      df.write
+        .format("carbondata")
+        .option("tableName", "carbon2")
+        .mode(SaveMode.Overwrite)
+        .save()
+
+      assert(DataMapWriterSuite.callbackSeq.head.contains("block start"))
+      assert(DataMapWriterSuite.callbackSeq.last.contains("block end"))
+      assert(
+        DataMapWriterSuite.callbackSeq.slice(1, DataMapWriterSuite.callbackSeq.length - 1) == Seq(
+          "blocklet start 0",
+          "add page data: blocklet 0, page 0",
+          "add page data: blocklet 0, page 1",
+          "add page data: blocklet 0, page 2",
+          "add page data: blocklet 0, page 3",
+          "add page data: blocklet 0, page 4",
+          "add page data: blocklet 0, page 5",
+          "add page data: blocklet 0, page 6",
+          "add page data: blocklet 0, page 7",
+          "blocklet end: 0",
+          "blocklet start 1",
+          "add page data: blocklet 1, page 0",
+          "add page data: blocklet 1, page 1",
+          "blocklet end: 1"
+        ))
+      DataMapWriterSuite.callbackSeq = Seq()
   }
 
   override def afterAll {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
index a16c5bf..2c6178d 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
@@ -24,8 +24,9 @@ import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsa
 import org.apache.commons.lang3.time.DateUtils
 import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterAll
-import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+
+import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
@@ -36,28 +37,16 @@ import org.apache.spark.sql.test.util.QueryTest
  */
 class DataRetentionTestCase extends QueryTest with BeforeAndAfterAll {
 
-  val absoluteTableIdentifierForLock: AbsoluteTableIdentifier = new
-      AbsoluteTableIdentifier(storeLocation,
-        new CarbonTableIdentifier(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "retentionlock", "200"))
-  val absoluteTableIdentifierForRetention: AbsoluteTableIdentifier = new
-      AbsoluteTableIdentifier(storeLocation,
-        new CarbonTableIdentifier(
-          CarbonCommonConstants.DATABASE_DEFAULT_NAME, "DataRetentionTable".toLowerCase(), "300"))
-  val carbonTablePath = CarbonStorePath
-    .getCarbonTablePath(absoluteTableIdentifierForRetention.getStorePath,
-      absoluteTableIdentifierForRetention.getCarbonTableIdentifier).getMetadataDirectoryPath
-
+  var absoluteTableIdentifierForLock: AbsoluteTableIdentifier = null
+  var absoluteTableIdentifierForRetention: AbsoluteTableIdentifier = null
+  var carbonTablePath : String = null
   var carbonDateFormat = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP)
   var defaultDateFormat = new SimpleDateFormat(CarbonCommonConstants
     .CARBON_TIMESTAMP_DEFAULT_FORMAT)
-  val carbonTableStatusLock: ICarbonLock = CarbonLockFactory
-    .getCarbonLockObj(absoluteTableIdentifierForLock.getCarbonTableIdentifier, LockUsage.TABLE_STATUS_LOCK)
-  val carbonDeleteSegmentLock: ICarbonLock = CarbonLockFactory
-    .getCarbonLockObj(absoluteTableIdentifierForLock.getCarbonTableIdentifier, LockUsage.DELETE_SEGMENT_LOCK)
-  val carbonCleanFilesLock: ICarbonLock = CarbonLockFactory
-    .getCarbonLockObj(absoluteTableIdentifierForLock.getCarbonTableIdentifier, LockUsage.CLEAN_FILES_LOCK)
-  val carbonMetadataLock: ICarbonLock = CarbonLockFactory
-    .getCarbonLockObj(absoluteTableIdentifierForLock.getCarbonTableIdentifier, LockUsage.METADATA_LOCK)
+  var carbonTableStatusLock: ICarbonLock = null
+  var carbonDeleteSegmentLock: ICarbonLock = null
+  var carbonCleanFilesLock: ICarbonLock = null
+  var carbonMetadataLock: ICarbonLock = null
 
   override def beforeAll {
     sql("drop table if exists DataRetentionTable")
@@ -78,7 +67,20 @@ class DataRetentionTestCase extends QueryTest with BeforeAndAfterAll {
       "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format'"
 
     )
-
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME+"_"+ "retentionlock")
+    absoluteTableIdentifierForLock = carbonTable.getAbsoluteTableIdentifier
+    val carbonTable2 = CarbonMetadata.getInstance().getCarbonTable(CarbonCommonConstants.DATABASE_DEFAULT_NAME+"_"+ "dataRetentionTable")
+    absoluteTableIdentifierForRetention = carbonTable2.getAbsoluteTableIdentifier
+    carbonTablePath = CarbonStorePath
+      .getCarbonTablePath(absoluteTableIdentifierForRetention).getMetadataDirectoryPath
+    carbonTableStatusLock = CarbonLockFactory
+      .getCarbonLockObj(absoluteTableIdentifierForLock, LockUsage.TABLE_STATUS_LOCK)
+    carbonDeleteSegmentLock= CarbonLockFactory
+      .getCarbonLockObj(absoluteTableIdentifierForLock, LockUsage.DELETE_SEGMENT_LOCK)
+    carbonCleanFilesLock = CarbonLockFactory
+      .getCarbonLockObj(absoluteTableIdentifierForLock, LockUsage.CLEAN_FILES_LOCK)
+    carbonMetadataLock = CarbonLockFactory
+      .getCarbonLockObj(absoluteTableIdentifierForLock, LockUsage.METADATA_LOCK)
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/dataretention1.csv' INTO TABLE retentionlock " +
       "OPTIONS('DELIMITER' =  ',')")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dblocation/DBLocationCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dblocation/DBLocationCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dblocation/DBLocationCarbonTableTestCase.scala
new file mode 100644
index 0000000..6f3191d
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dblocation/DBLocationCarbonTableTestCase.scala
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.spark.testsuite.dblocation
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+/**
+ *
+ */
+class DBLocationCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
+
+  def getMdtFileAndType() = {
+    // if mdt file path is configured then take configured path else take default path
+    val configuredMdtPath = CarbonProperties.getInstance()
+      .getProperty(CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER,
+        CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER_DEFAULT).trim
+    var timestampFile = configuredMdtPath + "/" + CarbonCommonConstants.SCHEMAS_MODIFIED_TIME_FILE
+    timestampFile = CarbonUtil.checkAndAppendFileSystemURIScheme(timestampFile)
+    val timestampFileType = FileFactory.getFileType(timestampFile)
+    (timestampFile, timestampFileType)
+
+  }
+
+  override def beforeAll {
+    sql("drop database if exists carbon cascade")
+  }
+
+  test("create and drop database test") {
+    sql(s"create database carbon location '$dblocation'")
+    sql("drop database if exists carbon cascade")
+  }
+
+  test("create two databases at same table") {
+    sql(s"create database carbon location '$dblocation'")
+    try {
+      sql(s"create database carbon1 location '$dblocation'")
+    } catch {
+      case e: AnalysisException =>
+        assert(true)
+    }
+  }
+
+  test("create table and load data") {
+    sql("drop database if exists carbon cascade")
+    sql(s"create database carbon location '$dblocation'")
+    sql("use carbon")
+    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/dblocation/test.csv' INTO table carbon.carbontable""")
+    checkAnswer(sql("select count(*) from carbontable"), Row(5))
+  }
+
+  test("create table and insert data") {
+    sql("drop database if exists carbon cascade")
+    sql(s"create database carbon location '$dblocation'")
+    sql("use carbon")
+    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql("insert into carbontable select 'a',1,'aa','aaa'")
+    checkAnswer(sql("select count(*) from carbontable"), Row(1))
+    checkAnswer(sql("select c1 from carbontable"), Seq(Row("a")))
+  }
+
+  test("create table and 2 times data load") {
+    sql("drop database if exists carbon cascade")
+    sql(s"create database carbon location '$dblocation'")
+    sql("use carbon")
+    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql("insert into carbontable select 'a',1,'aa','aaa'")
+    sql("insert into carbontable select 'b',1,'aa','aaa'")
+    checkAnswer(sql("select count(*) from carbontable"), Row(2))
+    checkAnswer(sql("select c1 from carbontable"), Seq(Row("a"), Row("b")))
+  }
+
+  //TODO fix this test case
+  test("Update operation on carbon table with insert into") {
+    sql("drop database if exists carbon cascade")
+    sql(s"create database carbon location '$dblocation'")
+    sql("use carbon")
+    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql("insert into carbontable select 'a',1,'aa','aaa'")
+    sql("insert into carbontable select 'b',1,'bb','bbb'")
+    // update operation
+    sql("""update carbon.carbontable d  set (d.c2) = (d.c2 + 1) where d.c1 = 'a'""").show()
+    sql("""update carbon.carbontable d  set (d.c2) = (d.c2 + 1) where d.c1 = 'b'""").show()
+    checkAnswer(
+      sql("""select c1,c2,c3,c5 from carbon.carbontable"""),
+      Seq(Row("a",2,"aa","aaa"),Row("b",2,"bb","bbb"))
+    )
+  }
+
+
+  test("Update operation on carbon table") {
+    sql("drop database if exists carbon cascade")
+    sql(s"create database carbon location '$dblocation'")
+    sql("use carbon")
+    sql(
+      """
+         CREATE TABLE automerge(id int, name string, city string, age int)
+         STORED BY 'org.apache.carbondata.format'
+      """)
+    val testData = s"$resourcesPath/sample.csv"
+    sql(s"LOAD DATA LOCAL INPATH '$testData' into table automerge")
+    // update operation
+    sql("""update carbon.automerge d  set (d.id) = (d.id + 1) where d.id > 2""").show()
+    checkAnswer(
+      sql("select count(*) from automerge"),
+      Seq(Row(6))
+    )
+    //    sql("drop table carbontable")
+  }
+
+  test("Delete operation on carbon table") {
+    sql("drop database if exists carbon cascade")
+    sql(s"create database carbon location '$dblocation'")
+    sql("use carbon")
+    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql("insert into carbontable select 'a',1,'aa','aaa'")
+    sql("insert into carbontable select 'b',1,'bb','bbb'")
+    // delete operation
+    sql("""delete from carbontable where c3 = 'aa'""").show
+    checkAnswer(
+      sql("""select c1,c2,c3,c5 from carbon.carbontable"""),
+      Seq(Row("b",1,"bb","bbb"))
+    )
+    sql("drop table carbontable")
+  }
+
+  test("Alter table add column test") {
+    sql("drop database if exists carbon cascade")
+    sql(s"create database carbon location '$dblocation'")
+    sql("use carbon")
+    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql("insert into carbontable select 'a',1,'aa','aaa'")
+    sql("insert into carbontable select 'b',1,'bb','bbb'")
+    sql("Alter table carbontable add columns(c4 string) " +
+        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='c4', 'DEFAULT.VALUE.c4'='def')")
+    checkAnswer(
+      sql("""select c1,c2,c3,c5,c4 from carbon.carbontable"""),
+      Seq(Row("a",1,"aa","aaa","def"), Row("b",1,"bb","bbb","def"))
+    )
+    sql("drop table carbontable")
+  }
+
+  test("Alter table change column datatype test") {
+    sql("drop database if exists carbon cascade")
+    sql(s"create database carbon location '$dblocation'")
+    sql("use carbon")
+    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql("insert into carbontable select 'a',1,'aa','aaa'")
+    sql("insert into carbontable select 'b',1,'bb','bbb'")
+    sql("Alter table carbontable change c2 c2 long")
+    checkAnswer(
+      sql("""select c1,c2,c3,c5 from carbon.carbontable"""),
+      Seq(Row("a",1,"aa","aaa"), Row("b",1,"bb","bbb"))
+    )
+    sql("drop table carbontable")
+  }
+
+  test("Alter table drop column test") {
+    sql("drop database if exists carbon cascade")
+    sql(s"create database carbon location '$dblocation'")
+    sql("use carbon")
+    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql("insert into carbontable select 'a',1,'aa','aaa'")
+    sql("insert into carbontable select 'b',1,'bb','bbb'")
+    sql("Alter table carbontable drop columns(c2)")
+    checkAnswer(
+      sql("""select * from carbon.carbontable"""),
+      Seq(Row("a","aa","aaa"), Row("b","bb","bbb"))
+    )
+    sql("drop table carbontable")
+  }
+
+  test("test mdt file path with configured paths") {
+    sql("drop database if exists carbon cascade")
+    sql(s"create database carbon location '$dblocation'")
+    sql("use carbon")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER, "/tmp/carbondata1/carbondata2/")
+    val (timestampFile, timestampFileType) = getMdtFileAndType()
+    FileFactory.deleteFile(timestampFile, timestampFileType)
+    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql("drop table carbontable")
+    // perform file check
+    assert(FileFactory.isFileExist(timestampFile, timestampFileType, true))
+
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER,
+        CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER_DEFAULT)
+    val (timestampFile2, timestampFileType2) = getMdtFileAndType()
+    FileFactory.deleteFile(timestampFile2, timestampFileType2)
+    sql("""create table carbon.carbontable (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql("drop table carbontable")
+    // perform file check
+    assert(FileFactory.isFileExist(timestampFile, timestampFileType, true))
+  }
+
+  override def afterAll {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER,
+        CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER_DEFAULT)
+    sql("use default")
+    sql("drop database if exists carbon cascade")
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
index 39777de..ed151bd 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
@@ -61,8 +61,8 @@ class TestDataLoadingForPartitionTable extends QueryTest with BeforeAndAfterAll
 
   def validateDataFiles(tableUniqueName: String, segmentId: String, partitions: Seq[Int]): Unit = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
-    val tablePath = new CarbonTablePath(carbonTable.getStorePath, carbonTable.getDatabaseName,
-      carbonTable.getFactTableName)
+    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
+      carbonTable.getTablePath)
     val segmentDir = tablePath.getCarbonDataDirectoryPath("0", segmentId)
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
index cf6a032..3d93a65 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
@@ -105,21 +105,26 @@ object CarbonStore {
       forceTableClean: Boolean): Unit = {
     LOGGER.audit(s"The clean files request has been received for $dbName.$tableName")
     var carbonCleanFilesLock: ICarbonLock = null
-    val identifier = new CarbonTableIdentifier(dbName, tableName, "")
+    var absoluteTableIdentifier: AbsoluteTableIdentifier = null
+    if (forceTableClean) {
+      absoluteTableIdentifier = AbsoluteTableIdentifier.from(storePath, dbName, tableName)
+    } else {
+      absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+    }
     try {
       val errorMsg = "Clean files request is failed for " +
                      s"$dbName.$tableName" +
                      ". Not able to acquire the clean files lock due to another clean files " +
                      "operation is running in the background."
       carbonCleanFilesLock =
-        CarbonLockUtil.getLockObject(identifier, LockUsage.CLEAN_FILES_LOCK, errorMsg)
+        CarbonLockUtil.getLockObject(absoluteTableIdentifier, LockUsage.CLEAN_FILES_LOCK, errorMsg)
       if (forceTableClean) {
         val absIdent = AbsoluteTableIdentifier.from(storePath, dbName, tableName)
         FileFactory.deleteAllCarbonFilesOfDir(
           FileFactory.getCarbonFile(absIdent.getTablePath,
             FileFactory.getFileType(absIdent.getTablePath)))
       } else {
-        DataManagementFunc.deleteLoadsAndUpdateMetadata(dbName, tableName, storePath,
+        DataManagementFunc.deleteLoadsAndUpdateMetadata(
           isForceDeletion = true, carbonTable)
         CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, true)
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
index 2fbc7a3..c86f7d3 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
@@ -23,7 +23,7 @@ import org.apache.spark.rdd.RDD
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.statusmanager.SegmentStatus
@@ -51,8 +51,7 @@ class AddColumnPartition(rddId: Int, idx: Int, schema: ColumnSchema) extends Par
  */
 class AlterTableAddColumnRDD[K, V](sc: SparkContext,
     @transient newColumns: Seq[ColumnSchema],
-    carbonTableIdentifier: CarbonTableIdentifier,
-    carbonStorePath: String)
+    absoluteTableIdentifier: AbsoluteTableIdentifier)
   extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
 
   val lockType: String = CarbonProperties.getInstance.getProperty(CarbonCommonConstants.LOCK_TYPE,
@@ -75,7 +74,7 @@ class AlterTableAddColumnRDD[K, V](sc: SparkContext,
         if (columnSchema.hasEncoding(Encoding.DICTIONARY) &&
             !columnSchema.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
           val carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(carbonStorePath, carbonTableIdentifier)
+            .getCarbonTablePath(absoluteTableIdentifier)
           var rawData: String = null
           if (null != columnSchema.getDefaultValue) {
             rawData = new String(columnSchema.getDefaultValue,
@@ -91,8 +90,7 @@ class AlterTableAddColumnRDD[K, V](sc: SparkContext,
           GlobalDictionaryUtil
             .loadDefaultDictionaryValueForNewColumn(carbonTablePath,
               columnSchema,
-              carbonTableIdentifier,
-              carbonStorePath,
+              absoluteTableIdentifier,
               rawData)
         }
       } catch {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
index 5b0889f..248f351 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
@@ -18,12 +18,11 @@
 package org.apache.carbondata.spark.rdd
 
 import org.apache.spark.{Partition, SparkContext, TaskContext}
-import org.apache.spark.rdd.RDD
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.cache.dictionary.ManageDictionaryAndBTree
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.statusmanager.SegmentStatus
@@ -48,8 +47,7 @@ class DropColumnPartition(rddId: Int, idx: Int, schema: ColumnSchema) extends Pa
  */
 class AlterTableDropColumnRDD[K, V](sc: SparkContext,
     @transient newColumns: Seq[ColumnSchema],
-    carbonTableIdentifier: CarbonTableIdentifier,
-    carbonStorePath: String)
+    carbonTableIdentifier: AbsoluteTableIdentifier)
   extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
 
   override def getPartitions: Array[Partition] = {
@@ -68,7 +66,7 @@ class AlterTableDropColumnRDD[K, V](sc: SparkContext,
         if (columnSchema.hasEncoding(Encoding.DICTIONARY) &&
             !columnSchema.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
           ManageDictionaryAndBTree
-            .deleteDictionaryFileAndCache(columnSchema, carbonTableIdentifier, carbonStorePath)
+            .deleteDictionaryFileAndCache(columnSchema, carbonTableIdentifier)
         }
       } catch {
         case ex: Exception =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
index c1c7bf1..e7a4a1c 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
@@ -38,7 +38,7 @@ import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumn
 import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
-import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
 import org.apache.carbondata.core.service.{CarbonCommonFactory, PathService}
 import org.apache.carbondata.core.statusmanager.SegmentStatus
@@ -151,7 +151,7 @@ case class DataFormat(delimiters: Array[String],
 /**
  * a case class to package some attributes
  */
-case class DictionaryLoadModel(table: CarbonTableIdentifier,
+case class DictionaryLoadModel(table: AbsoluteTableIdentifier,
     dimensions: Array[CarbonDimension],
     hdfsLocation: String,
     dictfolderPath: String,
@@ -280,8 +280,6 @@ class CarbonBlockDistinctValuesCombineRDD(
   override def internalCompute(split: Partition,
       context: TaskContext): Iterator[(Int, ColumnDistinctValues)] = {
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION,
-      model.hdfsLocation)
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordLoadCsvfilesToDfTime()
     val distinctValuesList = new ArrayBuffer[(Int, mutable.HashSet[String])]
     var rowCount = 0L
@@ -341,8 +339,6 @@ class CarbonGlobalDictionaryGenerateRDD(
   override def internalCompute(split: Partition,
       context: TaskContext): Iterator[(Int, SegmentStatus)] = {
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION,
-      model.hdfsLocation)
     var status = SegmentStatus.SUCCESS
     val iter = new Iterator[(Int, SegmentStatus)] {
       var dictionaryForDistinctValueLookUp: Dictionary = _
@@ -353,11 +349,11 @@ class CarbonGlobalDictionaryGenerateRDD(
         model.table,
         model.columnIdentifier(split.index),
         model.columnIdentifier(split.index).getDataType,
-        CarbonStorePath.getCarbonTablePath(model.hdfsLocation, model.table))
+        CarbonStorePath.getCarbonTablePath(model.table))
       val pathService: PathService = CarbonCommonFactory.getPathService
       val carbonTablePath: CarbonTablePath =
         pathService
-          .getCarbonTablePath(model.hdfsLocation, model.table, dictionaryColumnUniqueIdentifier)
+          .getCarbonTablePath(model.table, dictionaryColumnUniqueIdentifier)
       if (StringUtils.isNotBlank(model.hdfsTempLocation)) {
         CarbonProperties.getInstance.addProperty(CarbonCommonConstants.HDFS_TEMP_LOCATION,
           model.hdfsTempLocation)
@@ -371,7 +367,7 @@ class CarbonGlobalDictionaryGenerateRDD(
           model.zooKeeperUrl)
       }
       val dictLock = CarbonLockFactory
-        .getCarbonLockObj(carbonTablePath.getRelativeDictionaryDirectory,
+        .getCarbonLockObj(model.table,
           model.columnIdentifier(split.index).getColumnId + LockUsage.LOCK)
       var isDictionaryLocked = false
       // generate distinct value list
@@ -405,7 +401,6 @@ class CarbonGlobalDictionaryGenerateRDD(
         dictionaryForDistinctValueLookUp = if (isDictFileExists) {
           CarbonLoaderUtil.getDictionary(model.table,
             model.columnIdentifier(split.index),
-            model.hdfsLocation,
             model.primDimensions(split.index).getDataType
           )
         } else {
@@ -415,9 +410,7 @@ class CarbonGlobalDictionaryGenerateRDD(
         val t3 = System.currentTimeMillis()
         val dictWriteTask = new DictionaryWriterTask(valuesBuffer,
           dictionaryForDistinctValueLookUp,
-          model.table,
           dictionaryColumnUniqueIdentifier,
-          model.hdfsLocation,
           model.primDimensions(split.index).getColumnSchema,
           isDictFileExists
         )
@@ -427,10 +420,8 @@ class CarbonGlobalDictionaryGenerateRDD(
         val t4 = System.currentTimeMillis()
         // if new data came than rewrite sort index file
         if (distinctValues.size() > 0) {
-          val sortIndexWriteTask = new SortIndexWriterTask(model.table,
-            dictionaryColumnUniqueIdentifier,
+          val sortIndexWriteTask = new SortIndexWriterTask(dictionaryColumnUniqueIdentifier,
             model.primDimensions(split.index).getDataType,
-            model.hdfsLocation,
             dictionaryForDistinctValueLookUp,
             distinctValues)
           sortIndexWriteTask.execute()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index c6edc2a..2a7ca47 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -167,7 +167,7 @@ class CarbonMergerRDD[K, V](
         val dataFileMetadataSegMapping: java.util.Map[String, List[DataFileFooter]] =
           CarbonCompactionUtil.createDataFileFooterMappingForSegments(tableBlockInfoList)
 
-        carbonLoadModel.setStorePath(hdfsStoreLocation)
+        carbonLoadModel.setTablePath(hdfsStoreLocation)
         // check for restructured block
         // TODO: only in case of add and drop this variable should be true
         val restructuredBlockExists: Boolean = CarbonCompactionUtil

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
index 80cbab8..13a872f 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
@@ -54,12 +54,12 @@ import org.apache.carbondata.processing.util.CarbonLoaderUtil
  * This RDD is used in alter table partition statement to get data of target partitions,
  * then repartition data according to new partitionInfo
  * @param alterPartitionModel
- * @param carbonTableIdentifier
+ * @param absoluteTableIdentifier
  * @param partitionIds  the ids of target partition to be scanned
  * @param bucketId
  */
 class CarbonScanPartitionRDD(alterPartitionModel: AlterPartitionModel,
-    carbonTableIdentifier: CarbonTableIdentifier,
+    absoluteTableIdentifier: AbsoluteTableIdentifier,
     partitionIds: Seq[String],
     bucketId: Int)
   extends RDD[(AnyRef, Array[AnyRef])](alterPartitionModel.sqlContext.sparkContext, Nil) {
@@ -69,15 +69,14 @@ class CarbonScanPartitionRDD(alterPartitionModel: AlterPartitionModel,
   val segmentId = alterPartitionModel.segmentId
   val carbonLoadModel = alterPartitionModel.carbonLoadModel
   val oldPartitionIdList = alterPartitionModel.oldPartitionIds
-  val storePath = carbonLoadModel.getStorePath
-  val identifier = new AbsoluteTableIdentifier(storePath, carbonTableIdentifier)
   var storeLocation: String = null
   var splitStatus: Boolean = false
   var blockId: String = null
   val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
   val dimensions = carbonTable.getAllDimensions.asScala
   val measures = carbonTable.getAllMeasures.asScala
-  val partitionInfo = carbonTable.getPartitionInfo(carbonTableIdentifier.getTableName)
+  val partitionInfo = carbonTable
+    .getPartitionInfo(absoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
   val partitionColumn = partitionInfo.getColumnSchemaList().get(0)
   val partitionDataType = partitionColumn.getDataType
   val partitionColumnName = partitionColumn.getColumnName
@@ -94,7 +93,7 @@ class CarbonScanPartitionRDD(alterPartitionModel: AlterPartitionModel,
     val parallelism = sparkContext.defaultParallelism
     val jobConf = new JobConf(new Configuration)
     val job = new Job(jobConf)
-    val format = CarbonInputFormatUtil.createCarbonTableInputFormat(identifier,
+    val format = CarbonInputFormatUtil.createCarbonTableInputFormat(absoluteTableIdentifier,
       partitionIds.toList.asJava, job)
     job.getConfiguration.set("query.id", queryId)
 
@@ -114,7 +113,7 @@ class CarbonScanPartitionRDD(alterPartitionModel: AlterPartitionModel,
           val splits = blocksPerTask.asScala.map(_.asInstanceOf[CarbonInputSplit])
           if (blocksPerTask.size() != 0) {
             val multiBlockSplit =
-              new CarbonMultiBlockSplit(identifier, splits.asJava, Array(node))
+              new CarbonMultiBlockSplit(absoluteTableIdentifier, splits.asJava, Array(node))
             val partition = new CarbonSparkPartition(id, partition_num, multiBlockSplit)
             result.add(partition)
             partition_num += 1
@@ -150,8 +149,8 @@ class CarbonScanPartitionRDD(alterPartitionModel: AlterPartitionModel,
               sys.error("Exception occurred in query execution. Please check logs.")
             }
         }
-        val segmentProperties = PartitionUtils.getSegmentProperties(identifier, segmentId,
-          partitionIds.toList, oldPartitionIdList, partitionInfo)
+        val segmentProperties = PartitionUtils.getSegmentProperties(absoluteTableIdentifier,
+          segmentId, partitionIds.toList, oldPartitionIdList, partitionInfo)
         val partColIdx = getPartitionColumnIndex(partitionColumnName, segmentProperties)
         indexInitialise()
         for (iterator <- result.asScala) {
@@ -202,8 +201,8 @@ class CarbonScanPartitionRDD(alterPartitionModel: AlterPartitionModel,
           partitionValue = partitionValue.toString
         }
       } else {  // normal dictionary
-        val dict = CarbonLoaderUtil.getDictionary(carbonTableIdentifier,
-          dimension.getColumnIdentifier, storePath, partitionDataType)
+        val dict = CarbonLoaderUtil.getDictionary(absoluteTableIdentifier,
+          dimension.getColumnIdentifier, partitionDataType)
         if (partitionDataType == DataTypes.STRING) {
           if (partitionType == PartitionType.RANGE) {
             partitionValue = ByteUtil.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 52a31a9..a84b040 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -328,12 +328,21 @@ class CarbonScanRDD(
 
   private def prepareInputFormatForDriver(conf: Configuration): CarbonTableInputFormat[Object] = {
     CarbonTableInputFormat.setTableInfo(conf, tableInfo)
+    CarbonTableInputFormat.setDatabaseName(conf, tableInfo.getDatabaseName)
+    CarbonTableInputFormat
+      .setTableName(conf,
+        tableInfo.getOrCreateAbsoluteTableIdentifier().getCarbonTableIdentifier.getTableName)
     createInputFormat(conf)
   }
 
   private def prepareInputFormatForExecutor(conf: Configuration): CarbonTableInputFormat[Object] = {
     CarbonTableInputFormat.setCarbonReadSupport(conf, readSupport)
-    CarbonTableInputFormat.setTableInfo(conf, getTableInfo)
+    val tableInfo1 = getTableInfo
+    CarbonTableInputFormat.setTableInfo(conf, tableInfo1)
+    CarbonTableInputFormat.setDatabaseName(conf, tableInfo1.getDatabaseName)
+    CarbonTableInputFormat
+      .setTableName(conf,
+        tableInfo1.getOrCreateAbsoluteTableIdentifier().getCarbonTableIdentifier.getTableName)
     CarbonTableInputFormat.setDataTypeConverter(conf, new SparkDataTypeConverterImpl)
     createInputFormat(conf)
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
index adc71cf..a787af2 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
@@ -38,13 +38,12 @@ object Compactor {
 
   def triggerCompaction(compactionCallableModel: CompactionCallableModel): Unit = {
 
-    val storeLocation = compactionCallableModel.storeLocation
     val carbonTable = compactionCallableModel.carbonTable
     val loadsToMerge = compactionCallableModel.loadsToMerge
     val sc = compactionCallableModel.sqlContext
     val carbonLoadModel = compactionCallableModel.carbonLoadModel
     val compactionType = compactionCallableModel.compactionType
-    val storePath = carbonLoadModel.getStorePath
+    val storePath = carbonLoadModel.getTablePath
     val startTime = System.nanoTime()
     val mergedLoadName = CarbonDataMergerUtil.getMergedLoadName(loadsToMerge)
     var finalMergeStatus = false
@@ -53,8 +52,7 @@ object Compactor {
     val validSegments: Array[String] = CarbonDataMergerUtil
       .getValidSegments(loadsToMerge).split(',')
     val mergeLoadStartTime = CarbonUpdateUtil.readCurrentTime()
-    val carbonMergerMapping = CarbonMergerMapping(storeLocation,
-      storePath,
+    val carbonMergerMapping = CarbonMergerMapping(storePath,
       carbonTable.getMetaDataFilepath,
       mergedLoadName,
       databaseName,
@@ -65,7 +63,7 @@ object Compactor {
       maxSegmentColCardinality = null,
       maxSegmentColumnSchemaList = null
     )
-    carbonLoadModel.setStorePath(carbonMergerMapping.hdfsStoreLocation)
+    carbonLoadModel.setTablePath(carbonMergerMapping.hdfsStoreLocation)
     carbonLoadModel.setLoadMetadataDetails(
       SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath).toList.asJava)
     // trigger event for compaction

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
index 3a74dfa..762dcee 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
@@ -76,8 +76,7 @@ object DataManagementFunc {
         executor,
         sqlContext,
         compactionModel,
-        carbonLoadModel,
-        storeLocation
+        carbonLoadModel
       )
 
       try {
@@ -128,8 +127,7 @@ object DataManagementFunc {
       executor: ExecutorService,
       sqlContext: SQLContext,
       compactionModel: CompactionModel,
-      carbonLoadModel: CarbonLoadModel,
-      storeLocation: String
+      carbonLoadModel: CarbonLoadModel
   ): Unit = {
     loadsToMerge.asScala.foreach { seg =>
       LOGGER.info("loads identified for merge is " + seg.getLoadName)
@@ -137,7 +135,6 @@ object DataManagementFunc {
 
     val compactionCallableModel = CompactionCallableModel(
       carbonLoadModel,
-      storeLocation,
       compactionModel.carbonTable,
       loadsToMerge,
       sqlContext,
@@ -175,25 +172,21 @@ object DataManagementFunc {
   }
 
   def deleteLoadsAndUpdateMetadata(
-      dbName: String,
-      tableName: String,
-      storePath: String,
       isForceDeletion: Boolean,
       carbonTable: CarbonTable): Unit = {
     if (isLoadDeletionRequired(carbonTable.getMetaDataFilepath)) {
       val details = SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath)
+      val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
       val carbonTableStatusLock =
         CarbonLockFactory.getCarbonLockObj(
-          new CarbonTableIdentifier(dbName, tableName, ""),
+          absoluteTableIdentifier,
           LockUsage.TABLE_STATUS_LOCK
         )
 
       // Delete marked loads
       val isUpdationRequired =
         DeleteLoadFolders.deleteLoadFoldersFromFileSystem(
-          dbName,
-          tableName,
-          storePath,
+          absoluteTableIdentifier,
           isForceDeletion,
           details
         )
@@ -212,8 +205,10 @@ object DataManagementFunc {
             val latestStatus = CarbonLoaderUtil
                 .updateLoadMetadataFromOldToNew(details, latestMetadata)
 
-            CarbonLoaderUtil.writeLoadMetadata(storePath, dbName, tableName, latestStatus)
+            CarbonLoaderUtil.writeLoadMetadata(absoluteTableIdentifier, latestStatus)
           } else {
+            val dbName = absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName
+            val tableName = absoluteTableIdentifier.getCarbonTableIdentifier.getTableName
             val errorMsg = "Clean files request is failed for " +
                 s"$dbName.$tableName" +
                 ". Not able to acquire the table status lock due to other operation " +

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
index 0a41f44..a82ea00 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
@@ -42,8 +42,7 @@ object PartitionDropper {
     val carbonTable = dropPartitionCallableModel.carbonTable
     val dbName = carbonTable.getDatabaseName
     val tableName = carbonTable.getFactTableName
-    val identifier = carbonTable.getAbsoluteTableIdentifier
-    val carbonTableIdentifier = identifier.getCarbonTableIdentifier
+    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
     val partitionInfo = carbonTable.getPartitionInfo(tableName)
     val partitioner = PartitionFactory.getPartitioner(partitionInfo)
 
@@ -68,7 +67,7 @@ object PartitionDropper {
         for (i <- 0 until bucketNumber) {
           val bucketId = i
           val rdd = new CarbonScanPartitionRDD(alterPartitionModel,
-            carbonTableIdentifier,
+            absoluteTableIdentifier,
             Seq(partitionId, targetPartitionId),
             bucketId
           ).partitionBy(partitioner).map(_._2)
@@ -77,7 +76,7 @@ object PartitionDropper {
             new AlterPartitionResultImpl(),
             Seq(partitionId),
             bucketId,
-            identifier,
+            absoluteTableIdentifier,
             rdd).collect()
 
           if (dropStatus.length == 0) {
@@ -95,7 +94,7 @@ object PartitionDropper {
 
         if (finalDropStatus) {
           try {
-            PartitionUtils.deleteOriginalCarbonFile(alterPartitionModel, identifier,
+            PartitionUtils.deleteOriginalCarbonFile(alterPartitionModel, absoluteTableIdentifier,
               Seq(partitionId, targetPartitionId).toList, dbName,
               tableName, partitionInfo)
           } catch {
@@ -111,7 +110,7 @@ object PartitionDropper {
         case e: Exception => sys.error(s"Exception in dropping partition action: ${ e.getMessage }")
       }
     } else {
-      PartitionUtils.deleteOriginalCarbonFile(alterPartitionModel, identifier,
+      PartitionUtils.deleteOriginalCarbonFile(alterPartitionModel, absoluteTableIdentifier,
         Seq(partitionId).toList, dbName, tableName, partitionInfo)
       logger.audit(s"Drop Partition request completed for table " +
                    s"${ dbName }.${ tableName }")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
index fca7542..db664b3 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
@@ -39,8 +39,7 @@ object PartitionSplitter {
      val partitionId = splitPartitionCallableModel.partitionId
      val carbonLoadModel = splitPartitionCallableModel.carbonLoadModel
      val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-     val identifier = carbonTable.getAbsoluteTableIdentifier
-     val carbonTableIdentifier = identifier.getCarbonTableIdentifier
+     val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
      val tableName = carbonTable.getFactTableName
      val databaseName = carbonTable.getDatabaseName
      val bucketInfo = carbonTable.getBucketingInfo(tableName)
@@ -56,7 +55,7 @@ object PartitionSplitter {
        val bucketId = i
        val rdd = new CarbonScanPartitionRDD(
          alterPartitionModel,
-         carbonTableIdentifier,
+         absoluteTableIdentifier,
          Seq(partitionId),
          bucketId
        ).partitionBy(partitioner).map(_._2)
@@ -65,7 +64,7 @@ object PartitionSplitter {
          new AlterPartitionResultImpl(),
          Seq(partitionId),
          bucketId,
-         identifier,
+         absoluteTableIdentifier,
          rdd).collect()
 
        if (splitStatus.length == 0) {
@@ -83,8 +82,8 @@ object PartitionSplitter {
      if (finalSplitStatus) {
        try {
          PartitionUtils.
-           deleteOriginalCarbonFile(alterPartitionModel, identifier, Seq(partitionId).toList
-             , databaseName, tableName, partitionInfo)
+           deleteOriginalCarbonFile(alterPartitionModel, absoluteTableIdentifier,
+             Seq(partitionId).toList, databaseName, tableName, partitionInfo)
        } catch {
          case e: IOException => sys.error(s"Exception while delete original carbon files " +
          e.getMessage)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/DictionaryWriterTask.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/DictionaryWriterTask.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/DictionaryWriterTask.scala
index a9ac9f1..e304d84 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/DictionaryWriterTask.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/DictionaryWriterTask.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable
 
 import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.service.CarbonCommonFactory
 import org.apache.carbondata.core.util.DataTypeUtil
@@ -32,18 +32,14 @@ import org.apache.carbondata.core.writer.CarbonDictionaryWriter
  *
  * @param valuesBuffer
  * @param dictionary
- * @param carbonTableIdentifier
  * @param dictionaryColumnUniqueIdentifier
- * @param carbonStoreLocation
  * @param columnSchema
  * @param isDictionaryFileExist
  * @param writer
  */
 class DictionaryWriterTask(valuesBuffer: mutable.HashSet[String],
     dictionary: Dictionary,
-    carbonTableIdentifier: CarbonTableIdentifier,
     dictionaryColumnUniqueIdentifier: DictionaryColumnUniqueIdentifier,
-    carbonStoreLocation: String,
     columnSchema: ColumnSchema,
     isDictionaryFileExist: Boolean,
     var writer: CarbonDictionaryWriter = null) {
@@ -57,10 +53,7 @@ class DictionaryWriterTask(valuesBuffer: mutable.HashSet[String],
     val values = valuesBuffer.toArray
     java.util.Arrays.sort(values, Ordering[String])
     val dictService = CarbonCommonFactory.getDictionaryService
-    writer = dictService.getDictionaryWriter(
-      carbonTableIdentifier,
-      dictionaryColumnUniqueIdentifier,
-      carbonStoreLocation)
+    writer = dictService.getDictionaryWriter(dictionaryColumnUniqueIdentifier)
     val distinctValues: java.util.List[String] = new java.util.ArrayList()
 
     try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/SortIndexWriterTask.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/SortIndexWriterTask.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/SortIndexWriterTask.scala
index 27f9418..f212120 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/SortIndexWriterTask.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/SortIndexWriterTask.scala
@@ -25,19 +25,15 @@ import org.apache.carbondata.core.writer.sortindex.{CarbonDictionarySortIndexWri
 /**
  * This task writes sort index file
  *
- * @param carbonTableIdentifier
  * @param dictionaryColumnUniqueIdentifier
  * @param dataType
- * @param carbonStoreLocation
  * @param dictionary
  * @param distinctValues
  * @param carbonDictionarySortIndexWriter
  */
 class SortIndexWriterTask(
-    carbonTableIdentifier: CarbonTableIdentifier,
     dictionaryColumnUniqueIdentifier: DictionaryColumnUniqueIdentifier,
     dataType: DataType,
-    carbonStoreLocation: String,
     dictionary: Dictionary,
     distinctValues: java.util.List[String],
     var carbonDictionarySortIndexWriter: CarbonDictionarySortIndexWriter = null) {
@@ -51,8 +47,7 @@ class SortIndexWriterTask(
             dataType)
         carbonDictionarySortIndexWriter =
           dictService
-            .getDictionarySortIndexWriter(carbonTableIdentifier, dictionaryColumnUniqueIdentifier,
-            carbonStoreLocation)
+            .getDictionarySortIndexWriter(dictionaryColumnUniqueIdentifier)
         carbonDictionarySortIndexWriter.writeSortIndex(dictionarySortInfo.getSortIndex)
         carbonDictionarySortIndexWriter
           .writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index b51d0f0..1b21e3d 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -27,6 +27,7 @@ import scala.collection.mutable.Map
 
 import org.apache.commons.lang3.StringUtils
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
 import org.apache.spark.SparkContext
 import org.apache.spark.sql.{Row, RowFactory, SQLContext}
@@ -40,7 +41,7 @@ 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.memory.{UnsafeMemoryManager, UnsafeSortMemoryManager}
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.schema.PartitionInfo
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
@@ -54,6 +55,7 @@ import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.merger.TableMeta
 import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.rdd.CarbonMergeFilesRDD
@@ -848,14 +850,32 @@ object CommonUtil {
    */
   def mergeIndexFiles(sparkContext: SparkContext,
       segmentIds: Seq[String],
-      storePath: String,
+      tablePath: String,
       carbonTable: CarbonTable): Unit = {
     if (CarbonProperties.getInstance().getProperty(
       CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT,
       CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT).toBoolean) {
-      new CarbonMergeFilesRDD(sparkContext, AbsoluteTableIdentifier.from(storePath,
+      new CarbonMergeFilesRDD(sparkContext, AbsoluteTableIdentifier.from(tablePath,
         carbonTable.getDatabaseName, carbonTable.getFactTableName).getTablePath,
         segmentIds).collect()
     }
   }
+
+  /**
+   * can be removed with the spark 1.6 removal
+   * @param tableMeta
+   * @return
+   */
+  @deprecated
+  def getTablePath(tableMeta: TableMeta): String = {
+    if (tableMeta.tablePath == null) {
+      tableMeta.storePath + CarbonCommonConstants.FILE_SEPARATOR +
+      tableMeta.carbonTableIdentifier.getDatabaseName +
+      CarbonCommonConstants.FILE_SEPARATOR + tableMeta.carbonTableIdentifier.getTableName
+    }
+    else {
+      tableMeta.tablePath
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
index 5a24d7d..35e1e78 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
@@ -182,7 +182,7 @@ object DataLoadingUtil {
       carbonLoadModel: CarbonLoadModel): Unit = {
     carbonLoadModel.setTableName(table.getFactTableName)
     carbonLoadModel.setDatabaseName(table.getDatabaseName)
-    carbonLoadModel.setStorePath(table.getStorePath)
+    carbonLoadModel.setTablePath(table.getTablePath)
     carbonLoadModel.setTableName(table.getFactTableName)
     val dataLoadSchema = new CarbonDataLoadSchema(table)
     // Need to fill dimension relation

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index 3bb8d94..975fc9b 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -42,7 +42,7 @@ import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumn
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
-import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.column.{CarbonDimension, ColumnSchema}
@@ -174,12 +174,9 @@ object GlobalDictionaryUtil {
       model.table,
       model.columnIdentifier(columnIndex),
       model.columnIdentifier(columnIndex).getDataType,
-      CarbonStorePath.getCarbonTablePath(model.hdfsLocation, model.table))
-    val writer: CarbonDictionaryWriter = dictService.getDictionaryWriter(
-      model.table,
-      dictionaryColumnUniqueIdentifier,
-      model.hdfsLocation
-    )
+      CarbonStorePath.getCarbonTablePath(model.table))
+    val writer: CarbonDictionaryWriter = dictService
+      .getDictionaryWriter(dictionaryColumnUniqueIdentifier)
     try {
       while (iter.hasNext) {
         writer.write(iter.next)
@@ -196,8 +193,7 @@ object GlobalDictionaryUtil {
     val dictMap = new HashMap[String, Dictionary]
     model.primDimensions.zipWithIndex.filter(f => model.dictFileExists(f._2)).foreach { m =>
       val dict = CarbonLoaderUtil.getDictionary(model.table,
-        m._1.getColumnIdentifier, model.hdfsLocation,
-        m._1.getDataType
+        m._1.getColumnIdentifier, m._1.getDataType
       )
       dictMap.put(m._1.getColumnId, dict)
     }
@@ -218,12 +214,11 @@ object GlobalDictionaryUtil {
             model.table,
             model.columnIdentifier(i),
             model.columnIdentifier(i).getDataType,
-            CarbonStorePath.getCarbonTablePath(model.hdfsLocation, model.table))
+            CarbonStorePath.getCarbonTablePath(model.table))
       val set = new HashSet[String]
       if (model.dictFileExists(i)) {
-        val reader: CarbonDictionaryReader = dictService.getDictionaryReader(model.table,
-          dictionaryColumnUniqueIdentifier, model.hdfsLocation
-        )
+        val reader: CarbonDictionaryReader = dictService.getDictionaryReader(
+          dictionaryColumnUniqueIdentifier)
         val values = reader.read
         if (values != null) {
           for (j <- 0 until values.size) {
@@ -330,7 +325,8 @@ object GlobalDictionaryUtil {
     if (null == carbonLoadModel.getLoadMetadataDetails) {
       CommonUtil.readLoadMetadataDetails(carbonLoadModel)
     }
-    DictionaryLoadModel(table,
+    val absoluteTableIdentifier = new AbsoluteTableIdentifier(hdfsLocation, table)
+    DictionaryLoadModel(absoluteTableIdentifier,
       dimensions,
       hdfsLocation,
       dictfolderPath,
@@ -391,7 +387,7 @@ object GlobalDictionaryUtil {
       model: DictionaryLoadModel,
       status: Array[(Int, SegmentStatus)]) = {
     var result = false
-    val tableName = model.table.getTableName
+    val tableName = model.table.getCarbonTableIdentifier.getTableName
     status.foreach { x =>
       val columnName = model.primDimensions(x._1).getColName
       if (SegmentStatus.LOAD_FAILURE == x._2) {
@@ -819,21 +815,19 @@ object GlobalDictionaryUtil {
    *
    * @param carbonTablePath
    * @param columnSchema
-   * @param tableIdentifier
-   * @param storePath
+   * @param absoluteTableIdentifier
    * @param defaultValue
    */
   def loadDefaultDictionaryValueForNewColumn(carbonTablePath: CarbonTablePath,
       columnSchema: ColumnSchema,
-      tableIdentifier: CarbonTableIdentifier,
-      storePath: String,
+      absoluteTableIdentifier: AbsoluteTableIdentifier,
       defaultValue: String): Unit = {
 
     var carbonDictionarySortIndexWriter: CarbonDictionarySortIndexWriter = null
     var dictionary: Dictionary = null
 
     val dictLock = CarbonLockFactory
-      .getCarbonLockObj(carbonTablePath.getRelativeDictionaryDirectory,
+      .getCarbonLockObj(absoluteTableIdentifier,
         columnSchema.getColumnUniqueId + LockUsage.LOCK)
     var isDictionaryLocked = false
     try {
@@ -852,7 +846,7 @@ object GlobalDictionaryUtil {
         columnSchema.getDataType)
       val dictionaryColumnUniqueIdentifier: DictionaryColumnUniqueIdentifier = new
           DictionaryColumnUniqueIdentifier(
-            tableIdentifier,
+            absoluteTableIdentifier,
             columnIdentifier,
             columnIdentifier.getDataType,
             carbonTablePath)
@@ -863,9 +857,7 @@ object GlobalDictionaryUtil {
       }
       val dictWriteTask = new DictionaryWriterTask(valuesBuffer,
         dictionary,
-        tableIdentifier,
         dictionaryColumnUniqueIdentifier,
-        storePath,
         columnSchema,
         false
       )
@@ -875,10 +867,9 @@ object GlobalDictionaryUtil {
       }")
 
       if (distinctValues.size() > 0) {
-        val sortIndexWriteTask = new SortIndexWriterTask(tableIdentifier,
+        val sortIndexWriteTask = new SortIndexWriterTask(
           dictionaryColumnUniqueIdentifier,
           columnSchema.getDataType,
-          storePath,
           dictionary,
           distinctValues)
         sortIndexWriteTask.execute()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index a2f3364..756de6b 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -103,8 +103,7 @@ case class DataLoadTableFileMapping(table: String, loadPath: String)
 
 case class ExecutionErrors(var failureCauses: FailureCauses, var errorMsg: String )
 
-case class CarbonMergerMapping(storeLocation: String,
-    hdfsStoreLocation: String,
+case class CarbonMergerMapping(hdfsStoreLocation: String,
     metadataFilePath: String,
     var mergedLoadName: String,
     databaseName: String,
@@ -136,7 +135,6 @@ case class CompactionModel(compactionSize: Long,
     isDDLTrigger: Boolean)
 
 case class CompactionCallableModel(carbonLoadModel: CarbonLoadModel,
-    storeLocation: String,
     carbonTable: CarbonTable,
     loadsToMerge: util.List[LoadMetadataDetails],
     sqlContext: SQLContext,
@@ -203,7 +201,7 @@ class AlterTableColumnSchemaGenerator(
     tableInfo: TableInfo,
     carbonTablePath: CarbonTablePath,
     tableIdentifier: CarbonTableIdentifier,
-    storePath: String, sc: SparkContext) {
+    sc: SparkContext) {
 
   val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
index d2b565f..4a91b47 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
@@ -51,6 +51,7 @@ object TestQueryExecutor {
   LOGGER.info(s"project path: $projectPath")
   val integrationPath = s"$projectPath/integration"
   val metastoredb = s"$integrationPath/spark-common/target"
+  val location = s"$integrationPath/spark-common/target/dbpath"
   val masterUrl = {
     val property = System.getProperty("spark.master.url")
     if (property == null) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
index 0a4231c..c01cfef 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
@@ -105,6 +105,7 @@ class QueryTest extends PlanTest {
     getProperty(CarbonCommonConstants.STORE_LOCATION)
   val resourcesPath = TestQueryExecutor.resourcesPath
   val integrationPath = TestQueryExecutor.integrationPath
+  val dblocation = TestQueryExecutor.location
 }
 
 object QueryTest {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
index dcda72b..82052aa 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
@@ -29,7 +29,7 @@ import org.apache.hadoop.mapreduce.Job
 import org.apache.spark.sql.execution.command.AlterPartitionModel
 
 import org.apache.carbondata.core.datastore.block.{SegmentProperties, TableBlockInfo}
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.schema.PartitionInfo
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.util.CarbonUtil
@@ -160,12 +160,13 @@ object PartitionUtils {
     val segmentId = alterPartitionModel.segmentId
     val oldPartitionIds = alterPartitionModel.oldPartitionIds
     val newTime = carbonLoadModel.getFactTimeStamp
-    val storePath = carbonLoadModel.getStorePath
+    val tablePath = carbonLoadModel.getTablePath
     val tableBlockInfoList =
       getPartitionBlockList(identifier, segmentId, partitionIds, oldPartitionIds,
         partitionInfo).asScala
     val pathList: util.List[String] = new util.ArrayList[String]()
-    val carbonTablePath = new CarbonTablePath(storePath, dbName, tableName)
+    val carbonTableIdentifier = new CarbonTableIdentifier(dbName, tableName, "")
+    val carbonTablePath = new CarbonTablePath(carbonTableIdentifier, tablePath)
     tableBlockInfoList.foreach{ tableBlockInfo =>
       val path = tableBlockInfo.getFilePath
       val timestamp = CarbonTablePath.DataFileUtil.getTimeStampFromFileName(path)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index d7986dc..7c5599b 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -50,7 +50,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.scan.partition.PartitionUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties}
+import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.events.{LoadTablePostExecutionEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.exception.DataLoadingException
@@ -82,10 +82,15 @@ object CarbonDataRDDFactory {
       compactionType: CompactionType,
       carbonTable: CarbonTable,
       compactionModel: CompactionModel): Unit = {
+    // taking system level lock at the mdt file location
+    var configuredMdtPath = CarbonProperties.getInstance()
+      .getProperty(CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER,
+        CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER_DEFAULT).trim
+    configuredMdtPath = CarbonUtil.checkAndAppendFileSystemURIScheme(configuredMdtPath)
     val lock = CarbonLockFactory
-        .getCarbonLockObj(CarbonCommonConstants.SYSTEM_LEVEL_COMPACTION_LOCK_FOLDER,
-          LockUsage.SYSTEMLEVEL_COMPACTION_LOCK
-        )
+      .getCarbonLockObj(configuredMdtPath + CarbonCommonConstants.FILE_SEPARATOR +
+                        CarbonCommonConstants.SYSTEM_LEVEL_COMPACTION_LOCK_FOLDER,
+        LockUsage.SYSTEMLEVEL_COMPACTION_LOCK)
     if (lock.lockWithRetries()) {
       LOGGER.info(s"Acquired the compaction lock for table ${ carbonLoadModel.getDatabaseName }" +
           s".${ carbonLoadModel.getTableName }")
@@ -249,7 +254,7 @@ object CarbonDataRDDFactory {
     loadModel.setCarbonDataLoadSchema(dataLoadSchema)
     loadModel.setTableName(table.getCarbonTableIdentifier.getTableName)
     loadModel.setDatabaseName(table.getCarbonTableIdentifier.getDatabaseName)
-    loadModel.setStorePath(table.getStorePath)
+    loadModel.setTablePath(table.getTablePath)
     CommonUtil.readLoadMetadataDetails(loadModel)
     val loadStartTime = CarbonUpdateUtil.readCurrentTime()
     loadModel.setFactTimeStamp(loadStartTime)
@@ -273,8 +278,7 @@ object CarbonDataRDDFactory {
     LOGGER.audit(s"Data load request has been received for table" +
                  s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
     // Check if any load need to be deleted before loading new data
-    DataManagementFunc.deleteLoadsAndUpdateMetadata(carbonLoadModel.getDatabaseName,
-      carbonLoadModel.getTableName, storePath, isForceDeletion = false, carbonTable)
+    DataManagementFunc.deleteLoadsAndUpdateMetadata(isForceDeletion = false, carbonTable)
     var status: Array[(String, (LoadMetadataDetails, ExecutionErrors))] = null
     var res: Array[List[(String, (LoadMetadataDetails, ExecutionErrors))]] = null
 
@@ -531,7 +535,7 @@ object CarbonDataRDDFactory {
         carbonTable.getMetaDataFilepath)
       val segmentIds = loadMetadataDetails.map(_.getLoadName)
       val segmentIdIndex = segmentIds.zipWithIndex.toMap
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonLoadModel.getStorePath,
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonLoadModel.getTablePath,
         carbonTable.getCarbonTableIdentifier)
       val segmentId2maxTaskNo = segmentIds.map { segId =>
         (segId, CarbonUpdateUtil.getLatestTaskIdForSegment(segId, carbonTablePath))
@@ -698,7 +702,7 @@ object CarbonDataRDDFactory {
         )
       } else {
         val lock = CarbonLockFactory.getCarbonLockObj(
-          carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+          carbonTable.getAbsoluteTableIdentifier,
           LockUsage.COMPACTION_LOCK)
 
         if (lock.lockWithRetries()) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
index 5dd5983..1e6a36e 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
@@ -46,10 +46,9 @@ object CarbonSparkUtil {
   }
 
   def createCarbonRelation(tableInfo: TableInfo, tablePath: String): CarbonRelation = {
-    val identifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
     val table = CarbonTable.buildFromTableInfo(tableInfo)
-    val meta = new TableMeta(identifier.getCarbonTableIdentifier,
-      identifier.getStorePath, tablePath, table)
+    val meta = new TableMeta(table.getCarbonTableIdentifier,
+      table.getTablePath, tablePath, table)
     CarbonRelation(tableInfo.getDatabaseName, tableInfo.getFactTable.getTableName,
       CarbonSparkUtil.createSparkMeta(table), meta)
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
index 4acb82c..48f1a09 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
@@ -25,6 +25,9 @@ import org.apache.spark.sql.hive.HiveSessionCatalog
 import org.apache.spark.sql.optimizer.CarbonDecoderRelation
 import org.apache.spark.sql.types.{StringType, TimestampType}
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 
 case class CarbonDictionaryCatalystDecoder(
@@ -70,6 +73,40 @@ object GetDB {
     dbName.getOrElse(
       sparkSession.sessionState.catalog.asInstanceOf[HiveSessionCatalog].getCurrentDatabase)
   }
+
+  /**
+   * The method returns the database location
+   * if carbon.storeLocation does  point to spark.sql.warehouse.dir then returns
+   * the database locationUri as database location else follows the old behaviour
+   * making database location from carbon fixed store and database name.
+   *
+   * @param dbName
+   * @param sparkSession
+   * @param fixedStorePath
+   * @return
+   */
+  def getDatabaseLocation(dbName: String, sparkSession: SparkSession,
+      fixedStorePath: String): String = {
+    var databaseLocation =
+      sparkSession.sessionState.catalog.asInstanceOf[HiveSessionCatalog].getDatabaseMetadata(dbName)
+        .locationUri
+    // for default database and db ends with .db
+    // check whether the carbon store and hive store is same or different.
+    if (dbName.equals("default") || databaseLocation.endsWith(".db")) {
+      val properties = CarbonProperties.getInstance()
+      val carbonStorePath = FileFactory
+        .getUpdatedFilePath(properties.getProperty(CarbonCommonConstants.STORE_LOCATION))
+      val hiveStorePath = FileFactory
+        .getUpdatedFilePath(sparkSession.conf.get("spark.sql.warehouse.dir"))
+      // if carbon.store does not point to spark.sql.warehouse.dir then follow the old table path
+      // format
+      if (!hiveStorePath.equals(carbonStorePath)) {
+        databaseLocation = fixedStorePath + CarbonCommonConstants.FILE_SEPARATOR + dbName
+      }
+    }
+
+    return FileFactory.getUpdatedFilePath(databaseLocation)
+  }
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1155d4d8/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
index 3fb65be..22933f2 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
@@ -47,7 +47,8 @@ case class CarbonDatasourceHadoopRelation(
     isSubquery: ArrayBuffer[Boolean] = new ArrayBuffer[Boolean]())
   extends BaseRelation with InsertableRelation {
 
-  lazy val identifier: AbsoluteTableIdentifier = AbsoluteTableIdentifier.fromTablePath(paths.head)
+  lazy val identifier: AbsoluteTableIdentifier = AbsoluteTableIdentifier.from(paths.head,
+    parameters("dbname"), parameters("tablename"))
   lazy val databaseName: String = carbonTable.getDatabaseName
   lazy val tableName: String = carbonTable.getFactTableName
   CarbonSession.updateSessionInfoToCurrentThread(sparkSession)