You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2018/02/01 07:45:44 UTC

[1/4] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

Repository: carbondata
Updated Branches:
  refs/heads/carbonstore 15b4e192e -> c3e99681b


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 44204d4..f5a90de 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
@@ -32,8 +32,10 @@ import org.apache.spark.sql.types.StructType
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.{FileFormat, SegmentStatus}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
@@ -189,7 +191,6 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     val identifier = new TableIdentifier("batch_table", Option("streaming"))
     val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.lookupRelation(identifier)(spark)
       .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
     var server: ServerSocket = null
     try {
       server = getServerSocket
@@ -197,7 +198,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, server.getLocalPort, tablePath, identifier)
+      val thread2 = createSocketStreamingThread(spark, server.getLocalPort, carbonTable, identifier)
       val future = pool.submit(thread2)
       Thread.sleep(1000)
       thread1.interrupt()
@@ -220,11 +221,10 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     val identifier = new TableIdentifier("stream_table_file", Option("streaming"))
     val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.lookupRelation(identifier)(spark)
       .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
     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, carbonTable, csvDataDir, intervalSecond = 1,
       identifier)
     thread.start()
     Thread.sleep(2000)
@@ -646,12 +646,11 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     val identifier = new TableIdentifier("stream_table_drop", Option("streaming"))
     val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.lookupRelation(identifier)(spark)
       .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
     var server: ServerSocket = null
     try {
       server = getServerSocket
       val thread1 = createWriteSocketThread(server, 2, 10, 3)
-      val thread2 = createSocketStreamingThread(spark, server.getLocalPort, tablePath, identifier, "force", 5, 1024L * 200, false)
+      val thread2 = createSocketStreamingThread(spark, server.getLocalPort, carbonTable, identifier, "force", 5, 1024L * 200, false)
       thread1.start()
       thread2.start()
       Thread.sleep(1000)
@@ -749,7 +748,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
   def createSocketStreamingThread(
       spark: SparkSession,
       port: Int,
-      tablePath: CarbonTablePath,
+      carbonTable: CarbonTable,
       tableIdentifier: TableIdentifier,
       badRecordAction: String = "force",
       intervalSecond: Int = 2,
@@ -770,7 +769,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
           qry = readSocketDF.writeStream
             .format("carbondata")
             .trigger(ProcessingTime(s"$intervalSecond seconds"))
-            .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
+            .option("checkpointLocation", CarbonTablePath.getStreamingCheckpointDir(carbonTable.getTablePath))
             .option("bad_records_action", badRecordAction)
             .option("dbName", tableIdentifier.database.get)
             .option("tableName", tableIdentifier.table)
@@ -808,7 +807,6 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     val identifier = new TableIdentifier(tableName, Option("streaming"))
     val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.lookupRelation(identifier)(spark)
       .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
     var server: ServerSocket = null
     try {
       server = getServerSocket()
@@ -821,7 +819,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
       val thread2 = createSocketStreamingThread(
         spark = spark,
         port = server.getLocalPort,
-        tablePath = tablePath,
+        carbonTable = carbonTable,
         tableIdentifier = identifier,
         badRecordAction = badRecordAction,
         intervalSecond = intervalOfIngest,
@@ -863,7 +861,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
 
   def createFileStreamingThread(
       spark: SparkSession,
-      tablePath: CarbonTablePath,
+      carbonTable: CarbonTable,
       csvDataDir: String,
       intervalSecond: Int,
       tableIdentifier: TableIdentifier): Thread = {
@@ -889,7 +887,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
           qry = readSocketDF.writeStream
             .format("carbondata")
             .trigger(ProcessingTime(s"${ intervalSecond } seconds"))
-            .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
+            .option("checkpointLocation", CarbonTablePath.getStreamingCheckpointDir(carbonTable.getTablePath))
             .option("dbName", tableIdentifier.database.get)
             .option("tableName", tableIdentifier.table)
             .start()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
index 9a6efbe..97dc8ba 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
@@ -92,7 +92,7 @@ class AlterTableRevertTestCase extends Spark2QueryTest with BeforeAndAfterAll {
       }
       val carbonTable = CarbonMetadata.getInstance.getCarbonTable("default", "reverttest")
 
-      assert(new File(carbonTable.getMetaDataFilepath).listFiles().length < 6)
+      assert(new File(carbonTable.getMetadataPath).listFiles().length < 6)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java b/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
index a8db6c9..bbc3697 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
@@ -34,7 +34,6 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
 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.loading.model.CarbonLoadModel;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
@@ -56,43 +55,39 @@ public class TableProcessingOperations {
    */
   public static void deletePartialLoadDataIfExist(CarbonTable carbonTable,
       final boolean isCompactionFlow) throws IOException {
-    String metaDataLocation = carbonTable.getMetaDataFilepath();
+    String metaDataLocation = carbonTable.getMetadataPath();
     final LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(metaDataLocation);
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(carbonTable.getTablePath(), carbonTable.getCarbonTableIdentifier());
 
     //delete folder which metadata no exist in tablestatus
-    for (int i = 0; i < carbonTable.getPartitionCount(); i++) {
-      String partitionPath = carbonTablePath.getPartitionDir();
-      FileFactory.FileType fileType = FileFactory.getFileType(partitionPath);
-      if (FileFactory.isFileExist(partitionPath, fileType)) {
-        CarbonFile carbonFile = FileFactory.getCarbonFile(partitionPath, fileType);
-        CarbonFile[] listFiles = carbonFile.listFiles(new CarbonFileFilter() {
-          @Override public boolean accept(CarbonFile path) {
-            String segmentId =
-                CarbonTablePath.DataPathUtil.getSegmentId(path.getAbsolutePath() + "/dummy");
-            boolean found = false;
-            for (int j = 0; j < details.length; j++) {
-              if (details[j].getLoadName().equals(segmentId)) {
-                found = true;
-                break;
-              }
-            }
-            return !found;
-          }
-        });
-        for (int k = 0; k < listFiles.length; k++) {
+    String partitionPath = CarbonTablePath.getPartitionDir(carbonTable.getTablePath());
+    FileFactory.FileType fileType = FileFactory.getFileType(partitionPath);
+    if (FileFactory.isFileExist(partitionPath, fileType)) {
+      CarbonFile carbonFile = FileFactory.getCarbonFile(partitionPath, fileType);
+      CarbonFile[] listFiles = carbonFile.listFiles(new CarbonFileFilter() {
+        @Override public boolean accept(CarbonFile path) {
           String segmentId =
-              CarbonTablePath.DataPathUtil.getSegmentId(listFiles[k].getAbsolutePath() + "/dummy");
-          if (isCompactionFlow) {
-            if (segmentId.contains(".")) {
-              CarbonLoaderUtil.deleteStorePath(listFiles[k].getAbsolutePath());
-            }
-          } else {
-            if (!segmentId.contains(".")) {
-              CarbonLoaderUtil.deleteStorePath(listFiles[k].getAbsolutePath());
+              CarbonTablePath.DataFileUtil.getSegmentId(path.getAbsolutePath() + "/dummy");
+          boolean found = false;
+          for (int j = 0; j < details.length; j++) {
+            if (details[j].getLoadName().equals(segmentId)) {
+              found = true;
+              break;
             }
           }
+          return !found;
+        }
+      });
+      for (int k = 0; k < listFiles.length; k++) {
+        String segmentId =
+            CarbonTablePath.DataFileUtil.getSegmentId(listFiles[k].getAbsolutePath() + "/dummy");
+        if (isCompactionFlow) {
+          if (segmentId.contains(".")) {
+            CarbonLoaderUtil.deleteStorePath(listFiles[k].getAbsolutePath());
+          }
+        } else {
+          if (!segmentId.contains(".")) {
+            CarbonLoaderUtil.deleteStorePath(listFiles[k].getAbsolutePath());
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 4cd5014..193d192 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
@@ -34,8 +34,6 @@ 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;
@@ -105,12 +103,11 @@ public class FieldEncoderFactory {
           ColumnIdentifier parentColumnIdentifier =
               new ColumnIdentifier(parentColumnTableRelation.getColumnId(), null,
                   dataField.getColumn().getDataType());
-          CarbonTablePath carbonTablePath =
-              CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
           AbsoluteTableIdentifier parentAbsoluteTableIdentifier =
               AbsoluteTableIdentifier.from(
-              CarbonUtil.getNewTablePath(carbonTablePath, parentTableIdentifier.getTableName()),
-              parentTableIdentifier);
+                  CarbonTablePath.getNewTablePath(
+                      absoluteTableIdentifier.getTablePath(), parentTableIdentifier.getTableName()),
+                  parentTableIdentifier);
           identifier = new DictionaryColumnUniqueIdentifier(parentAbsoluteTableIdentifier,
               parentColumnIdentifier, dataField.getColumn().getDataType());
           return new DictionaryFieldConverterImpl(dataField, cache, parentAbsoluteTableIdentifier,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 d3caa99..a08177a 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
@@ -19,10 +19,8 @@ package org.apache.carbondata.processing.merger;
 
 import java.util.List;
 
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.scan.result.iterator.RawResultIterator;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
 import org.apache.carbondata.processing.store.CarbonDataFileAttributes;
 import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
@@ -42,13 +40,11 @@ public abstract class AbstractResultProcessor {
   public abstract boolean execute(List<RawResultIterator> resultIteratorList);
 
   protected void setDataFileAttributesInModel(CarbonLoadModel loadModel,
-      CompactionType compactionType, CarbonTable carbonTable,
-      CarbonFactDataHandlerModel carbonFactDataHandlerModel) {
+      CompactionType compactionType, CarbonFactDataHandlerModel carbonFactDataHandlerModel) {
     CarbonDataFileAttributes carbonDataFileAttributes;
     if (compactionType == CompactionType.IUD_UPDDEL_DELTA) {
       long taskNo = CarbonUpdateUtil.getLatestTaskIdForSegment(loadModel.getSegmentId(),
-          CarbonStorePath.getCarbonTablePath(loadModel.getTablePath(),
-              carbonTable.getCarbonTableIdentifier()));
+          loadModel.getTablePath());
       // 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.
       long index = taskNo + 1;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
index d796262..4fa4ff4 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
@@ -263,7 +263,7 @@ public class CarbonCompactionUtil {
   public static CarbonTable getNextTableToCompact(CarbonTable[] carbonTables,
       List<CarbonTableIdentifier> skipList) {
     for (CarbonTable ctable : carbonTables) {
-      String metadataPath = ctable.getMetaDataFilepath();
+      String metadataPath = ctable.getMetadataPath();
       // check for the compaction required file and at the same time exclude the tables which are
       // present in the skip list.
       if (CarbonCompactionUtil.isCompactionRequiredForTable(metadataPath) && !skipList

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 0eadc7f..c43dbf9 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
@@ -31,7 +31,6 @@ import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.locks.ICarbonLock;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.mutate.DeleteDeltaBlockDetails;
@@ -42,7 +41,6 @@ import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
 import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
@@ -167,15 +165,13 @@ public final class CarbonDataMergerUtil {
     // End Timestamp.
 
     // Table Update Status Metadata Update.
-    AbsoluteTableIdentifier absoluteTableIdentifier =
+    AbsoluteTableIdentifier identifier =
         carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
 
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-
     SegmentUpdateStatusManager segmentUpdateStatusManager =
-        new SegmentUpdateStatusManager(absoluteTableIdentifier);
+        new SegmentUpdateStatusManager(identifier);
 
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
 
     ICarbonLock updateLock = segmentUpdateStatusManager.getTableUpdateStatusLock();
     ICarbonLock statusLock = segmentStatusManager.getTableStatusLock();
@@ -222,7 +218,7 @@ public final class CarbonDataMergerUtil {
           }
 
           LoadMetadataDetails[] loadDetails =
-              segmentStatusManager.readLoadMetadata(metaDataFilepath);
+              SegmentStatusManager.readLoadMetadata(metaDataFilepath);
 
           for (LoadMetadataDetails loadDetail : loadDetails) {
             if (loadsToMerge.contains(loadDetail)) {
@@ -235,18 +231,18 @@ public final class CarbonDataMergerUtil {
             }
           }
 
-          segmentUpdateStatusManager
-              .writeLoadDetailsIntoFile(Arrays.asList(updateLists), timestamp);
-          segmentStatusManager
-              .writeLoadDetailsIntoFile(carbonTablePath.getTableStatusFilePath(), loadDetails);
+          segmentUpdateStatusManager.writeLoadDetailsIntoFile(
+              Arrays.asList(updateLists), timestamp);
+          SegmentStatusManager.writeLoadDetailsIntoFile(
+              CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()), loadDetails);
           status = true;
         } else {
           LOGGER.error("Not able to acquire the lock.");
           status = false;
         }
       } catch (IOException e) {
-        LOGGER.error("Error while updating metadata. The metadata file path is " + carbonTablePath
-            .getMetadataDirectoryPath());
+        LOGGER.error("Error while updating metadata. The metadata file path is " +
+            CarbonTablePath.getMetadataPath(identifier.getTablePath()));
         status = false;
 
       } finally {
@@ -282,9 +278,9 @@ public final class CarbonDataMergerUtil {
       String metaDataFilepath, String mergedLoadNumber, CarbonLoadModel carbonLoadModel,
       CompactionType compactionType) throws IOException {
     boolean tableStatusUpdationStatus = false;
-    AbsoluteTableIdentifier absoluteTableIdentifier =
+    AbsoluteTableIdentifier identifier =
         carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
 
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
 
@@ -293,10 +289,7 @@ public final class CarbonDataMergerUtil {
         LOGGER.info("Acquired lock for the table " + carbonLoadModel.getDatabaseName() + "."
             + carbonLoadModel.getTableName() + " for table status updation ");
 
-        CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier);
-
-        String statusFilePath = carbonTablePath.getTableStatusFilePath();
+        String statusFilePath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
 
         LoadMetadataDetails[] loadDetails = SegmentStatusManager.readLoadMetadata(metaDataFilepath);
 
@@ -595,10 +588,6 @@ public final class CarbonDataMergerUtil {
     List<LoadMetadataDetails> segmentsToBeMerged =
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
-    CarbonTableIdentifier tableIdentifier =
-        carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getCarbonTableIdentifier();
-
-
     // total length
     long totalLength = 0;
 
@@ -613,7 +602,7 @@ public final class CarbonDataMergerUtil {
       String segId = segment.getLoadName();
       // variable to store one  segment size across partition.
       long sizeOfOneSegmentAcrossPartition =
-          getSizeOfSegment(tablePath, tableIdentifier, segId);
+          getSizeOfSegment(tablePath, segId);
 
       // if size of a segment is greater than the Major compaction size. then ignore it.
       if (sizeOfOneSegmentAcrossPartition > (compactionSize * 1024 * 1024)) {
@@ -652,35 +641,17 @@ public final class CarbonDataMergerUtil {
   /**
    * For calculating the size of the specified segment
    * @param tablePath the store path of the segment
-   * @param tableIdentifier identifier of table that the segment belong to
    * @param segId segment id
    * @return the data size of the segment
    */
-  private static long getSizeOfSegment(String tablePath,
-      CarbonTableIdentifier tableIdentifier, String segId) {
-    String loadPath = getStoreLocation(tablePath, tableIdentifier, segId);
+  private static long getSizeOfSegment(String tablePath, String segId) {
+    String loadPath = CarbonTablePath.getSegmentPath(tablePath, segId);
     CarbonFile segmentFolder =
         FileFactory.getCarbonFile(loadPath, FileFactory.getFileType(loadPath));
     return getSizeOfFactFileInLoad(segmentFolder);
   }
 
   /**
-   * This method will get the store location for the given path, segemnt id and partition id
-   *
-   * @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 tablePath,
-      CarbonTableIdentifier carbonTableIdentifier, String segmentId) {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(tablePath, carbonTableIdentifier);
-    return carbonTablePath.getCarbonDataDirectoryPath(segmentId);
-  }
-
-
-  /**
    * Identify the segments to be merged based on the segment count
    *
    * @param listOfSegmentsAfterPreserve the list of segments after
@@ -1022,21 +993,19 @@ public final class CarbonDataMergerUtil {
    * if UpdateDelta Files are more than IUD Compaction threshold.
    *
    * @param seg
-   * @param absoluteTableIdentifier
+   * @param identifier
    * @param segmentUpdateStatusManager
    * @param numberDeltaFilesThreshold
    * @return
    */
   public static Boolean checkUpdateDeltaFilesInSeg(String seg,
-      AbsoluteTableIdentifier absoluteTableIdentifier,
+      AbsoluteTableIdentifier identifier,
       SegmentUpdateStatusManager segmentUpdateStatusManager, int numberDeltaFilesThreshold) {
 
     CarbonFile[] updateDeltaFiles = null;
     Set<String> uniqueBlocks = new HashSet<String>();
 
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(seg);
+    String segmentPath = CarbonTablePath.getSegmentPath(identifier.getTablePath(), seg);
     CarbonFile segDir =
         FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
     CarbonFile[] allSegmentFiles = segDir.listFiles();
@@ -1282,15 +1251,12 @@ public final class CarbonDataMergerUtil {
     CarbonUpdateUtil.updateSegmentStatus(segmentUpdateDetails, table, timestamp, true);
 
     // Update the Table Status.
-    String metaDataFilepath = table.getMetaDataFilepath();
-    AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier();
-
-    CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier);
+    String metaDataFilepath = table.getMetadataPath();
+    AbsoluteTableIdentifier identifier = table.getAbsoluteTableIdentifier();
 
-    String tableStatusPath = carbonTablePath.getTableStatusFilePath();
+    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
 
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
 
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
 
@@ -1304,7 +1270,7 @@ public final class CarbonDataMergerUtil {
                         + " for table status updation");
 
         LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-                segmentStatusManager.readLoadMetadata(metaDataFilepath);
+                SegmentStatusManager.readLoadMetadata(metaDataFilepath);
 
         for (LoadMetadataDetails loadMetadata : listOfLoadFolderDetailsArray) {
           if (loadMetadata.getLoadName().equalsIgnoreCase("0")) {
@@ -1313,7 +1279,7 @@ public final class CarbonDataMergerUtil {
           }
         }
         try {
-          segmentStatusManager
+          SegmentStatusManager
                   .writeLoadDetailsIntoFile(tableStatusPath, listOfLoadFolderDetailsArray);
         } catch (IOException e) {
           return false;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
index ff65db2..8fc6e66 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
@@ -404,8 +404,7 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
     CarbonFactDataHandlerModel carbonFactDataHandlerModel = CarbonFactDataHandlerModel
         .getCarbonFactDataHandlerModel(carbonLoadModel, carbonTable, segmentProperties, tableName,
             tempStoreLocation);
-    setDataFileAttributesInModel(carbonLoadModel, compactionType, carbonTable,
-        carbonFactDataHandlerModel);
+    setDataFileAttributesInModel(carbonLoadModel, compactionType, carbonFactDataHandlerModel);
     dataHandler = CarbonFactHandlerFactory.createCarbonFactHandler(carbonFactDataHandlerModel,
         CarbonFactHandlerFactory.FactHandlerType.COLUMNAR);
     try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
index 3d0700b..6f506b1 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
@@ -72,8 +72,7 @@ public class RowResultMergerProcessor extends AbstractResultProcessor {
     CarbonFactDataHandlerModel carbonFactDataHandlerModel = CarbonFactDataHandlerModel
         .getCarbonFactDataHandlerModel(loadModel, carbonTable, segProp, tableName,
             tempStoreLocation);
-    setDataFileAttributesInModel(loadModel, compactionType, carbonTable,
-        carbonFactDataHandlerModel);
+    setDataFileAttributesInModel(loadModel, compactionType, carbonFactDataHandlerModel);
     carbonFactDataHandlerModel.setCompactionFlow(true);
     dataHandler = new CarbonFactDataHandlerColumnar(carbonFactDataHandlerModel);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 9f3c86f..bc87823 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
@@ -35,7 +35,6 @@ 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.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.datamap.DataMapWriterListener;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
@@ -308,8 +307,7 @@ public class CarbonFactDataHandlerModel {
     }
     carbonFactDataHandlerModel.setMeasureDataType(measureDataTypes);
     String carbonDataDirectoryPath = CarbonDataProcessorUtil
-        .checkAndCreateCarbonStoreLocation(carbonTable.getTablePath(), loadModel.getDatabaseName(),
-            tableName, loadModel.getSegmentId());
+        .checkAndCreateCarbonStoreLocation(carbonTable.getTablePath(), loadModel.getSegmentId());
     carbonFactDataHandlerModel.setCarbonDataDirectoryPath(carbonDataDirectoryPath);
     List<CarbonDimension> dimensionByTableName = carbonTable.getDimensionByTableName(tableName);
     boolean[] isUseInvertedIndexes = new boolean[dimensionByTableName.size()];
@@ -334,10 +332,9 @@ public class CarbonFactDataHandlerModel {
    * @return data directory path
    */
   private static String getCarbonDataFolderLocation(CarbonDataLoadConfiguration configuration) {
-    AbsoluteTableIdentifier absoluteTableIdentifier = configuration.getTableIdentifier();
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
+    AbsoluteTableIdentifier identifier = configuration.getTableIdentifier();
     String carbonDataDirectoryPath =
-        carbonTablePath.getCarbonDataDirectoryPath(configuration.getSegmentId());
+        CarbonTablePath.getSegmentPath(identifier.getTablePath(), configuration.getSegmentId());
     CarbonUtil.checkAndCreateFolder(carbonDataDirectoryPath);
     return carbonDataDirectoryPath;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
index cfe6e31..ccde9e1 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -34,7 +34,6 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.ColumnType;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
@@ -44,7 +43,6 @@ 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.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;
 import org.apache.carbondata.processing.datatypes.ArrayDataType;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
@@ -143,12 +141,9 @@ public final class CarbonDataProcessorUtil {
     String[] baseTmpStorePathArray = StringUtils.split(baseTempStorePath, File.pathSeparator);
     String[] localDataFolderLocArray = new String[baseTmpStorePathArray.length];
 
-    CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
     for (int i = 0 ; i < baseTmpStorePathArray.length; i++) {
       String tmpStore = baseTmpStorePathArray[i];
-      CarbonTablePath carbonTablePath =
-          CarbonStorePath.getCarbonTablePath(tmpStore, carbonTable.getCarbonTableIdentifier());
-      String carbonDataDirectoryPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
+      String carbonDataDirectoryPath = CarbonTablePath.getSegmentPath(tmpStore, segmentId);
 
       localDataFolderLocArray[i] = carbonDataDirectoryPath + File.separator + taskId;
     }
@@ -375,12 +370,9 @@ public final class CarbonDataProcessorUtil {
    * @return data directory path
    */
   public static String checkAndCreateCarbonStoreLocation(String factStoreLocation,
-      String databaseName, String tableName, String segmentId) {
-    CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
-    CarbonTableIdentifier carbonTableIdentifier = carbonTable.getCarbonTableIdentifier();
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(factStoreLocation, carbonTableIdentifier);
-    String carbonDataDirectoryPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
+      String segmentId) {
+    String carbonDataDirectoryPath = CarbonTablePath.getSegmentPath(
+        factStoreLocation, segmentId);
     CarbonUtil.checkAndCreateFolder(carbonDataDirectoryPath);
     return carbonDataDirectoryPath;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 7be61d9..c2f4501 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
@@ -45,7 +45,6 @@ import org.apache.carbondata.core.fileoperations.FileWriteOperation;
 import org.apache.carbondata.core.locks.CarbonLockUtil;
 import org.apache.carbondata.core.locks.ICarbonLock;
 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;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -54,7 +53,6 @@ import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 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.loading.model.CarbonLoadModel;
 import org.apache.carbondata.processing.merger.NodeBlockRelation;
@@ -73,11 +71,8 @@ public final class CarbonLoaderUtil {
   }
 
   public static void deleteSegment(CarbonLoadModel loadModel, int currentLoad) {
-    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(loadModel.getTablePath(), carbonTable.getCarbonTableIdentifier());
-
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(currentLoad + "");
+    String segmentPath = CarbonTablePath.getSegmentPath(
+        loadModel.getTablePath(), currentLoad + "");
     deleteStorePath(segmentPath);
   }
 
@@ -90,33 +85,26 @@ public final class CarbonLoaderUtil {
    */
   public static boolean isValidSegment(CarbonLoadModel loadModel,
       int currentLoad) {
-    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema()
-        .getCarbonTable();
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(
-        loadModel.getTablePath(), carbonTable.getCarbonTableIdentifier());
 
     int fileCount = 0;
-    int partitionCount = carbonTable.getPartitionCount();
-    for (int i = 0; i < partitionCount; i++) {
-      String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(
-          currentLoad + "");
-      CarbonFile carbonFile = FileFactory.getCarbonFile(segmentPath,
-          FileFactory.getFileType(segmentPath));
-      CarbonFile[] files = carbonFile.listFiles(new CarbonFileFilter() {
-
-        @Override
-        public boolean accept(CarbonFile file) {
-          return file.getName().endsWith(
-              CarbonTablePath.getCarbonIndexExtension())
-              || file.getName().endsWith(
-              CarbonTablePath.getCarbonDataExtension());
-        }
-
-      });
-      fileCount += files.length;
-      if (files.length > 0) {
-        return true;
+    String segmentPath = CarbonTablePath.getSegmentPath(
+        loadModel.getTablePath(), currentLoad + "");
+    CarbonFile carbonFile = FileFactory.getCarbonFile(segmentPath,
+        FileFactory.getFileType(segmentPath));
+    CarbonFile[] files = carbonFile.listFiles(new CarbonFileFilter() {
+
+      @Override
+      public boolean accept(CarbonFile file) {
+        return file.getName().endsWith(
+            CarbonTablePath.getCarbonIndexExtension())
+            || file.getName().endsWith(
+            CarbonTablePath.getCarbonDataExtension());
       }
+
+    });
+    fileCount += files.length;
+    if (files.length > 0) {
+      return true;
     }
     if (fileCount == 0) {
       return false;
@@ -149,16 +137,15 @@ public final class CarbonLoaderUtil {
       CarbonLoadModel loadModel, boolean loadStartEntry, boolean insertOverwrite)
       throws IOException {
     boolean status = false;
-    AbsoluteTableIdentifier absoluteTableIdentifier =
+    AbsoluteTableIdentifier identifier =
         loadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    String metadataPath = carbonTablePath.getMetadataDirectoryPath();
+    String metadataPath = CarbonTablePath.getMetadataPath(identifier.getTablePath());
     FileType fileType = FileFactory.getFileType(metadataPath);
     if (!FileFactory.isFileExist(metadataPath, fileType)) {
       FileFactory.mkdirs(metadataPath, fileType);
     }
-    String tableStatusPath = carbonTablePath.getTableStatusFilePath();
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     int retryCount = CarbonLockUtil
         .getLockProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
@@ -172,7 +159,8 @@ public final class CarbonLoaderUtil {
             "Acquired lock for table" + loadModel.getDatabaseName() + "." + loadModel.getTableName()
                 + " for table status updation");
         LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-            SegmentStatusManager.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());
+            SegmentStatusManager.readLoadMetadata(
+                CarbonTablePath.getMetadataPath(identifier.getTablePath()));
         List<LoadMetadataDetails> listOfLoadFolderDetails =
             new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
         List<CarbonFile> staleFolders = new ArrayList<>();
@@ -198,13 +186,13 @@ public final class CarbonLoaderUtil {
           // is triggered
           for (LoadMetadataDetails entry : listOfLoadFolderDetails) {
             if (entry.getSegmentStatus() == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS
-                && segmentStatusManager.checkIfValidLoadInProgress(
-                    absoluteTableIdentifier, entry.getLoadName())) {
+                && SegmentStatusManager.checkIfValidLoadInProgress(
+                    identifier, entry.getLoadName())) {
               throw new RuntimeException("Already insert overwrite is in progress");
             } else if (newMetaEntry.getSegmentStatus() == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS
                 && entry.getSegmentStatus() == SegmentStatus.INSERT_IN_PROGRESS
-                && segmentStatusManager.checkIfValidLoadInProgress(
-                    absoluteTableIdentifier, entry.getLoadName())) {
+                && SegmentStatusManager.checkIfValidLoadInProgress(
+                identifier, entry.getLoadName())) {
               throw new RuntimeException("Already insert into or load is in progress");
             }
           }
@@ -227,7 +215,7 @@ public final class CarbonLoaderUtil {
                 entry.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE);
                 // For insert overwrite, we will delete the old segment folder immediately
                 // So collect the old segments here
-                addToStaleFolders(carbonTablePath, staleFolders, entry);
+                addToStaleFolders(identifier, staleFolders, entry);
               }
             }
           }
@@ -236,7 +224,7 @@ public final class CarbonLoaderUtil {
         // when no records are inserted then newSegmentEntry will be SegmentStatus.MARKED_FOR_DELETE
         // so empty segment folder should be deleted
         if (newMetaEntry.getSegmentStatus() == SegmentStatus.MARKED_FOR_DELETE) {
-          addToStaleFolders(carbonTablePath, staleFolders, newMetaEntry);
+          addToStaleFolders(identifier, staleFolders, newMetaEntry);
         }
 
         SegmentStatusManager.writeLoadDetailsIntoFile(tableStatusPath, listOfLoadFolderDetails
@@ -270,9 +258,10 @@ public final class CarbonLoaderUtil {
     return status;
   }
 
-  private static void addToStaleFolders(CarbonTablePath carbonTablePath,
+  private static void addToStaleFolders(AbsoluteTableIdentifier identifier,
       List<CarbonFile> staleFolders, LoadMetadataDetails entry) throws IOException {
-    String path = carbonTablePath.getCarbonDataDirectoryPath(entry.getLoadName());
+    String path = CarbonTablePath.getSegmentPath(
+        identifier.getTablePath(), entry.getLoadName());
     // add to the deletion list only if file exist else HDFS file system will throw
     // exception while deleting the file if file path does not exist
     if (FileFactory.isFileExist(path, FileFactory.getFileType(path))) {
@@ -298,11 +287,9 @@ public final class CarbonLoaderUtil {
     loadMetadataDetails.setLoadStartTime(loadStartTime);
   }
 
-  public static void writeLoadMetadata(AbsoluteTableIdentifier absoluteTableIdentifier,
+  public static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
       List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    String dataLoadLocation = carbonTablePath.getTableStatusFilePath();
+    String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
 
     DataOutputStream dataOutputStream;
     Gson gsonObjectToWrite = new Gson();
@@ -838,10 +825,8 @@ public final class CarbonLoaderUtil {
    * This method will get the store location for the given path, segment id and partition id
    */
   public static void checkAndCreateCarbonDataLocation(String segmentId, CarbonTable carbonTable) {
-    CarbonTableIdentifier carbonTableIdentifier = carbonTable.getCarbonTableIdentifier();
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath(), carbonTableIdentifier);
-    String segmentFolder = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
+    String segmentFolder = CarbonTablePath.getSegmentPath(
+        carbonTable.getTablePath(), segmentId);
     CarbonUtil.checkAndCreateFolder(segmentFolder);
   }
 
@@ -870,10 +855,8 @@ public final class CarbonLoaderUtil {
    */
   public static Long addDataIndexSizeIntoMetaEntry(LoadMetadataDetails loadMetadataDetails,
       String segmentId, CarbonTable carbonTable) throws IOException {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath((carbonTable.getAbsoluteTableIdentifier()));
     Map<String, Long> dataIndexSize =
-        CarbonUtil.getDataSizeAndIndexSize(carbonTablePath, segmentId);
+        CarbonUtil.getDataSizeAndIndexSize(carbonTable.getTablePath(), segmentId);
     Long dataSize = dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE);
     loadMetadataDetails.setDataSize(String.valueOf(dataSize));
     Long indexSize = dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_INDEX_SIZE);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 f9f3e20..1fdce32 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
@@ -32,7 +32,6 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 public final class DeleteLoadFolders {
@@ -47,15 +46,14 @@ public final class DeleteLoadFolders {
   /**
    * returns segment path
    *
-   * @param absoluteTableIdentifier
+   * @param identifier
    * @param oneLoad
    * @return
    */
-  private static String getSegmentPath(AbsoluteTableIdentifier absoluteTableIdentifier,
+  private static String getSegmentPath(AbsoluteTableIdentifier identifier,
       LoadMetadataDetails oneLoad) {
-    CarbonTablePath carbon = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     String segmentId = oneLoad.getLoadName();
-    return carbon.getCarbonDataDirectoryPath(segmentId);
+    return CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId);
   }
 
   public static void physicalFactAndMeasureMetadataDeletion(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 7925b35..e059b35 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
@@ -73,12 +73,12 @@ public class BlockIndexStoreTest extends TestCase {
 //            file.length(), ColumnarFormatVersion.V1, null);
 //    CarbonTableIdentifier carbonTableIdentifier =
 //            new CarbonTableIdentifier(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "t3", "1");
-//    AbsoluteTableIdentifier absoluteTableIdentifier =
+//    AbsoluteTableIdentifier identifier =
 //        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
 //    try {
 //
 //      List<TableBlockUniqueIdentifier> tableBlockInfoList =
-//          getTableBlockUniqueIdentifierList(Arrays.asList(new TableBlockInfo[] { info }), absoluteTableIdentifier);
+//          getTableBlockUniqueIdentifierList(Arrays.asList(new TableBlockInfo[] { info }), identifier);
 //      List<AbstractIndex> loadAndGetBlocks = cache.getAll(tableBlockInfoList);
 //      assertTrue(loadAndGetBlocks.size() == 1);
 //    } catch (Exception e) {
@@ -86,7 +86,7 @@ public class BlockIndexStoreTest extends TestCase {
 //    }
 //    List<String> segmentIds = new ArrayList<>();
 //      segmentIds.add(info.getSegmentId());
-//    cache.removeTableBlocks(segmentIds, absoluteTableIdentifier);
+//    cache.removeTableBlocks(segmentIds, identifier);
 //  }
 //
   private List<TableBlockUniqueIdentifier> getTableBlockUniqueIdentifierList(List<TableBlockInfo> tableBlockInfos,
@@ -122,19 +122,19 @@ public class BlockIndexStoreTest extends TestCase {
 //
 //    CarbonTableIdentifier carbonTableIdentifier =
 //            new CarbonTableIdentifier(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "t3", "1");
-//    AbsoluteTableIdentifier absoluteTableIdentifier =
+//    AbsoluteTableIdentifier identifier =
 //        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
 //    ExecutorService executor = Executors.newFixedThreadPool(3);
 //    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
-//        absoluteTableIdentifier));
+//        identifier));
 //    executor.submit(
 //        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
-//            absoluteTableIdentifier));
+//            identifier));
 //    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
-//        absoluteTableIdentifier));
+//        identifier));
 //    executor.submit(
 //        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
-//            absoluteTableIdentifier));
+//            identifier));
 //    executor.shutdown();
 //    try {
 //      executor.awaitTermination(1, TimeUnit.DAYS);
@@ -145,7 +145,7 @@ public class BlockIndexStoreTest extends TestCase {
 //        Arrays.asList(new TableBlockInfo[] { info, info1, info2, info3, info4 });
 //    try {
 //      List<TableBlockUniqueIdentifier> tableBlockUniqueIdentifiers =
-//          getTableBlockUniqueIdentifierList(tableBlockInfos, absoluteTableIdentifier);
+//          getTableBlockUniqueIdentifierList(tableBlockInfos, identifier);
 //      List<AbstractIndex> loadAndGetBlocks = cache.getAll(tableBlockUniqueIdentifiers);
 //      assertTrue(loadAndGetBlocks.size() == 5);
 //    } catch (Exception e) {
@@ -155,7 +155,7 @@ public class BlockIndexStoreTest extends TestCase {
 //    for (TableBlockInfo tableBlockInfo : tableBlockInfos) {
 //      segmentIds.add(tableBlockInfo.getSegmentId());
 //    }
-//    cache.removeTableBlocks(segmentIds, absoluteTableIdentifier);
+//    cache.removeTableBlocks(segmentIds, identifier);
 //  }
 //
 //  public void testloadAndGetTaskIdToSegmentsMapForDifferentSegmentLoadedConcurrently()
@@ -193,18 +193,18 @@ public class BlockIndexStoreTest extends TestCase {
 //
 //    CarbonTableIdentifier carbonTableIdentifier =
 //            new CarbonTableIdentifier(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "t3", "1");
-//    AbsoluteTableIdentifier absoluteTableIdentifier =
+//    AbsoluteTableIdentifier identifier =
 //        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
 //    ExecutorService executor = Executors.newFixedThreadPool(3);
 //    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
-//        absoluteTableIdentifier));
+//        identifier));
 //    executor.submit(
 //        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
-//            absoluteTableIdentifier));
+//            identifier));
 //    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info5, info6 }),
-//        absoluteTableIdentifier));
+//        identifier));
 //    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info7 }),
-//        absoluteTableIdentifier));
+//        identifier));
 //
 //    executor.shutdown();
 //    try {
@@ -217,7 +217,7 @@ public class BlockIndexStoreTest extends TestCase {
 //        .asList(new TableBlockInfo[] { info, info1, info2, info3, info4, info5, info6, info7 });
 //    try {
 //      List<TableBlockUniqueIdentifier> blockUniqueIdentifierList =
-//          getTableBlockUniqueIdentifierList(tableBlockInfos, absoluteTableIdentifier);
+//          getTableBlockUniqueIdentifierList(tableBlockInfos, identifier);
 //      List<AbstractIndex> loadAndGetBlocks = cache.getAll(blockUniqueIdentifierList);
 //      assertTrue(loadAndGetBlocks.size() == 8);
 //    } catch (Exception e) {
@@ -227,7 +227,7 @@ public class BlockIndexStoreTest extends TestCase {
 //    for (TableBlockInfo tableBlockInfo : tableBlockInfos) {
 //      segmentIds.add(tableBlockInfo.getSegmentId());
 //    }
-//    cache.removeTableBlocks(segmentIds, absoluteTableIdentifier);
+//    cache.removeTableBlocks(segmentIds, identifier);
 //  }
 
   private class BlockLoaderThread implements Callable<Void> {
@@ -250,7 +250,7 @@ public class BlockIndexStoreTest extends TestCase {
   }
 
   private static File getPartFile() {
-    String path = StoreCreator.getAbsoluteTableIdentifier().getTablePath()
+    String path = StoreCreator.getIdentifier().getTablePath()
         + "/Fact/Part0/Segment_0";
     File file = new File(path);
     File[] files = file.listFiles();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 7f0aef6..d42dcde 100644
--- a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
+++ b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
@@ -64,7 +64,6 @@ import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 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;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
@@ -98,14 +97,14 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
  */
 public class StoreCreator {
 
-  private static AbsoluteTableIdentifier absoluteTableIdentifier;
+  private static AbsoluteTableIdentifier identifier;
   private static String storePath = "";
   static {
     try {
       storePath = new File("target/store").getCanonicalPath();
       String dbName = "testdb";
       String tableName = "testtable";
-      absoluteTableIdentifier =
+      identifier =
           AbsoluteTableIdentifier.from(
               storePath + "/testdb/testtable",
               new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
@@ -114,8 +113,8 @@ public class StoreCreator {
     }
   }
 
-  public static AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    return absoluteTableIdentifier;
+  public static AbsoluteTableIdentifier getIdentifier() {
+    return identifier;
   }
 
   /**
@@ -134,12 +133,12 @@ public class StoreCreator {
       CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table);
       CarbonLoadModel loadModel = new CarbonLoadModel();
       loadModel.setCarbonDataLoadSchema(schema);
-      loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
-      loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
-      loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+      loadModel.setDatabaseName(identifier.getCarbonTableIdentifier().getDatabaseName());
+      loadModel.setTableName(identifier.getCarbonTableIdentifier().getTableName());
+      loadModel.setTableName(identifier.getCarbonTableIdentifier().getTableName());
       loadModel.setFactFilePath(factFilePath);
       loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
-      loadModel.setTablePath(absoluteTableIdentifier.getTablePath());
+      loadModel.setTablePath(identifier.getTablePath());
       loadModel.setDateFormat(null);
       loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
           CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
@@ -175,9 +174,9 @@ public class StoreCreator {
 
   private static CarbonTable createTable() throws IOException {
     TableInfo tableInfo = new TableInfo();
-    tableInfo.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
+    tableInfo.setDatabaseName(identifier.getCarbonTableIdentifier().getDatabaseName());
     TableSchema tableSchema = new TableSchema();
-    tableSchema.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+    tableSchema.setTableName(identifier.getCarbonTableIdentifier().getTableName());
     List<ColumnSchema> columnSchemas = new ArrayList<ColumnSchema>();
     ArrayList<Encoding> encodings = new ArrayList<>();
     encodings.add(Encoding.DICTIONARY);
@@ -257,16 +256,13 @@ public class StoreCreator {
     tableSchema.setSchemaEvalution(schemaEvol);
     tableSchema.setTableId(UUID.randomUUID().toString());
     tableInfo.setTableUniqueName(
-        absoluteTableIdentifier.getCarbonTableIdentifier().getTableUniqueName()
+        identifier.getCarbonTableIdentifier().getTableUniqueName()
     );
     tableInfo.setLastUpdatedTime(System.currentTimeMillis());
     tableInfo.setFactTable(tableSchema);
-    tableInfo.setTablePath(absoluteTableIdentifier.getTablePath());
+    tableInfo.setTablePath(identifier.getTablePath());
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-    String schemaFilePath = carbonTablePath.getSchemaFilePath();
+    String schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath());
     String schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath);
     CarbonMetadata.getInstance().loadTableMetadata(tableInfo);
 
@@ -329,7 +325,7 @@ public class StoreCreator {
       writer.close();
       writer.commit();
       Dictionary dict = (Dictionary) dictCache.get(
-          new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
+          new DictionaryColumnUniqueIdentifier(identifier,
         		  columnIdentifier, dims.get(i).getDataType()));
       CarbonDictionarySortInfoPreparator preparator =
           new CarbonDictionarySortInfoPreparator();
@@ -444,7 +440,7 @@ public class StoreCreator {
     loadMetadataDetails.setLoadStartTime(loadMetadataDetails.getTimeStamp(readCurrentTime()));
     listOfLoadFolderDetails.add(loadMetadataDetails);
 
-    String dataLoadLocation = schema.getCarbonTable().getMetaDataFilepath() + File.separator
+    String dataLoadLocation = schema.getCarbonTable().getMetadataPath() + File.separator
         + CarbonCommonConstants.LOADMETADATA_FILENAME;
 
     DataOutputStream dataOutputStream;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
index 7b823ac..8c9889d 100644
--- a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
@@ -38,7 +38,6 @@ import org.apache.carbondata.core.statusmanager.FileFormat;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.CarbonIndexFileWriter;
 import org.apache.carbondata.format.BlockIndex;
@@ -60,8 +59,6 @@ public class StreamSegment {
    * get stream segment or create new stream segment if not exists
    */
   public static String open(CarbonTable table) throws IOException {
-    CarbonTablePath tablePath =
-        CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier());
     SegmentStatusManager segmentStatusManager =
         new SegmentStatusManager(table.getAbsoluteTableIdentifier());
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
@@ -72,7 +69,8 @@ public class StreamSegment {
                 + " for stream table get or create segment");
 
         LoadMetadataDetails[] details =
-            SegmentStatusManager.readLoadMetadata(tablePath.getMetadataDirectoryPath());
+            SegmentStatusManager.readLoadMetadata(
+                CarbonTablePath.getMetadataPath(table.getTablePath()));
         LoadMetadataDetails streamSegment = null;
         for (LoadMetadataDetails detail : details) {
           if (FileFormat.ROW_V1 == detail.getFileFormat()) {
@@ -97,8 +95,8 @@ public class StreamSegment {
             newDetails[i] = details[i];
           }
           newDetails[i] = newDetail;
-          SegmentStatusManager
-              .writeLoadDetailsIntoFile(tablePath.getTableStatusFilePath(), newDetails);
+          SegmentStatusManager.writeLoadDetailsIntoFile(
+              CarbonTablePath.getTableStatusFilePath(table.getTablePath()), newDetails);
           return newDetail.getLoadName();
         } else {
           return streamSegment.getLoadName();
@@ -126,8 +124,6 @@ public class StreamSegment {
    */
   public static String close(CarbonTable table, String segmentId)
       throws IOException {
-    CarbonTablePath tablePath =
-        CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier());
     SegmentStatusManager segmentStatusManager =
         new SegmentStatusManager(table.getAbsoluteTableIdentifier());
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
@@ -138,7 +134,8 @@ public class StreamSegment {
                 + " for stream table finish segment");
 
         LoadMetadataDetails[] details =
-            SegmentStatusManager.readLoadMetadata(tablePath.getMetadataDirectoryPath());
+            SegmentStatusManager.readLoadMetadata(
+                CarbonTablePath.getMetadataPath(table.getTablePath()));
         for (LoadMetadataDetails detail : details) {
           if (segmentId.equals(detail.getLoadName())) {
             detail.setLoadEndTime(System.currentTimeMillis());
@@ -162,7 +159,8 @@ public class StreamSegment {
         }
         newDetails[i] = newDetail;
         SegmentStatusManager
-            .writeLoadDetailsIntoFile(tablePath.getTableStatusFilePath(), newDetails);
+            .writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(
+                table.getTablePath()), newDetails);
         return newDetail.getLoadName();
       } else {
         LOGGER.error(
@@ -192,7 +190,7 @@ public class StreamSegment {
     try {
       if (statusLock.lockWithRetries()) {
         LoadMetadataDetails[] details =
-            SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath());
+            SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
         boolean updated = false;
         for (LoadMetadataDetails detail : details) {
           if (SegmentStatus.STREAMING == detail.getSegmentStatus()) {
@@ -202,10 +200,8 @@ public class StreamSegment {
           }
         }
         if (updated) {
-          CarbonTablePath tablePath =
-              CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
           SegmentStatusManager.writeLoadDetailsIntoFile(
-              tablePath.getTableStatusFilePath(),
+              CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath()),
               details);
         }
       } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
index 197cb14..186d100 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
@@ -37,7 +37,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.scan.result.iterator.RawResultIterator
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
 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.hadoop.{CarbonInputSplit, CarbonProjection}
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
 import org.apache.carbondata.hadoop.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
@@ -216,7 +216,6 @@ object StreamHandoffRDD {
   ): Unit = {
     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
     val identifier = carbonTable.getAbsoluteTableIdentifier
-    val tablePath = CarbonStorePath.getCarbonTablePath(identifier)
     var continueHandoff = false
     // require handoff lock on table
     val lock = CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.HANDOFF_LOCK)
@@ -233,7 +232,7 @@ object StreamHandoffRDD {
           try {
             if (statusLock.lockWithRetries()) {
               loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
-                tablePath.getMetadataDirectoryPath)
+                CarbonTablePath.getMetadataPath(identifier.getTablePath))
             }
           } finally {
             if (null != statusLock) {
@@ -355,19 +354,16 @@ object StreamHandoffRDD {
       loadModel: CarbonLoadModel
   ): Boolean = {
     var status = false
-    val metaDataFilepath =
-      loadModel.getCarbonDataLoadSchema().getCarbonTable().getMetaDataFilepath()
-    val identifier =
-      loadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier()
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(identifier)
-    val metadataPath = carbonTablePath.getMetadataDirectoryPath()
+    val metaDataFilepath = loadModel.getCarbonDataLoadSchema.getCarbonTable.getMetadataPath
+    val identifier = loadModel.getCarbonDataLoadSchema.getCarbonTable.getAbsoluteTableIdentifier
+    val metadataPath = CarbonTablePath.getMetadataPath(identifier.getTablePath)
     val fileType = FileFactory.getFileType(metadataPath)
     if (!FileFactory.isFileExist(metadataPath, fileType)) {
       FileFactory.mkdirs(metadataPath, fileType)
     }
-    val tableStatusPath = carbonTablePath.getTableStatusFilePath()
+    val tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath)
     val segmentStatusManager = new SegmentStatusManager(identifier)
-    val carbonLock = segmentStatusManager.getTableStatusLock()
+    val carbonLock = segmentStatusManager.getTableStatusLock
     try {
       if (carbonLock.lockWithRetries()) {
         LOGGER.info(
@@ -400,7 +396,7 @@ object StreamHandoffRDD {
         status = true
       } else {
         LOGGER.error("Not able to acquire the lock for Table status updation for table " + loadModel
-          .getDatabaseName() + "." + loadModel.getTableName());
+          .getDatabaseName() + "." + loadModel.getTableName())
       }
     } finally {
       if (carbonLock.unlock()) {
@@ -411,6 +407,6 @@ object StreamHandoffRDD {
                      "." + loadModel.getTableName() + " during table status updation")
       }
     }
-    return status
+    status
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
index f2274be..c417fbe 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
@@ -31,7 +31,7 @@ import org.apache.carbondata.core.dictionary.service.NonSecureDictionaryServiceP
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
@@ -126,16 +126,14 @@ object StreamSinkFactory {
    * @return
    */
   private def getStreamSegmentId(carbonTable: CarbonTable): String = {
-    val carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val fileType = FileFactory.getFileType(carbonTablePath.getMetadataDirectoryPath)
-    if (!FileFactory.isFileExist(carbonTablePath.getMetadataDirectoryPath, fileType)) {
+    val segmentId = StreamSegment.open(carbonTable)
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
+    val fileType = FileFactory.getFileType(segmentDir)
+    if (!FileFactory.isFileExist(segmentDir, fileType)) {
       // Create table directory path, in case of enabling hive metastore first load may not have
       // table folder created.
-      FileFactory.mkdirs(carbonTablePath.getMetadataDirectoryPath, fileType)
+      FileFactory.mkdirs(segmentDir, fileType)
     }
-    val segmentId = StreamSegment.open(carbonTable)
-    val segmentDir = carbonTablePath.getSegmentDir(segmentId)
     if (FileFactory.isFileExist(segmentDir, fileType)) {
       // recover fault
       StreamSegment.recoverSegmentIfRequired(segmentDir)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
index 45bc19a..ff483e5 100644
--- a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
+++ b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
@@ -40,7 +40,7 @@ import org.apache.carbondata.core.dictionary.server.DictionaryServer
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.stats.QueryStatistic
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.hadoop.streaming.CarbonStreamOutputFormat
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
@@ -62,9 +62,7 @@ class CarbonAppendableStreamSink(
     carbonLoadModel: CarbonLoadModel,
     server: Option[DictionaryServer]) extends Sink {
 
-  private val carbonTablePath = CarbonStorePath
-    .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-  private val fileLogPath = carbonTablePath.getStreamingLogDir
+  private val fileLogPath = CarbonTablePath.getStreamingLogDir(carbonTable.getTablePath)
   private val fileLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, sparkSession, fileLogPath)
   // prepare configuration
   private val hadoopConf = {
@@ -149,12 +147,12 @@ class CarbonAppendableStreamSink(
    * if the directory size of current segment beyond the threshold, hand off new segment
    */
   private def checkOrHandOffSegment(): Unit = {
-    val segmentDir = carbonTablePath.getSegmentDir(currentSegmentId)
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, currentSegmentId)
     val fileType = FileFactory.getFileType(segmentDir)
     if (segmentMaxSize <= StreamSegment.size(segmentDir)) {
       val newSegmentId = StreamSegment.close(carbonTable, currentSegmentId)
       currentSegmentId = newSegmentId
-      val newSegmentDir = carbonTablePath.getSegmentDir(currentSegmentId)
+      val newSegmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, currentSegmentId)
       FileFactory.mkdirs(newSegmentDir, fileType)
 
       // TODO trigger hand off operation
@@ -250,15 +248,13 @@ object CarbonAppendableStreamSink {
         }
 
         // update data file info in index file
-        val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-        StreamSegment.updateIndexFile(tablePath.getSegmentDir(segmentId))
+        StreamSegment.updateIndexFile(
+          CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId))
 
       } catch {
         // catch fault of executor side
         case t: Throwable =>
-          val tablePath =
-            CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-          val segmentDir = tablePath.getSegmentDir(segmentId)
+          val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
           StreamSegment.recoverSegmentIfRequired(segmentDir)
           LOGGER.error(t, s"Aborting job ${ job.getJobID }.")
           committer.abortJob(job)


[2/4] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 d96a051..90a4223 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
@@ -49,7 +49,7 @@ import org.apache.carbondata.core.scan.partition.PartitionUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
 import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.comparator.Comparator
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
@@ -634,7 +634,7 @@ object CommonUtil {
 
 
   def readLoadMetadataDetails(model: CarbonLoadModel): Unit = {
-    val metadataPath = model.getCarbonDataLoadSchema.getCarbonTable.getMetaDataFilepath
+    val metadataPath = model.getCarbonDataLoadSchema.getCarbonTable.getMetadataPath
     val details = SegmentStatusManager.readLoadMetadata(metadataPath)
     model.setLoadMetadataDetails(new util.ArrayList[LoadMetadataDetails](details.toList.asJava))
   }
@@ -848,20 +848,18 @@ object CommonUtil {
       val fileType = FileFactory.getFileType(databaseLocation)
       if (FileFactory.isFileExist(databaseLocation, fileType)) {
         val file = FileFactory.getCarbonFile(databaseLocation, fileType)
-          if (file.isDirectory) {
-            val tableFolders = file.listFiles()
-            tableFolders.foreach { tableFolder =>
-              if (tableFolder.isDirectory) {
-                val tablePath = databaseLocation +
-                                CarbonCommonConstants.FILE_SEPARATOR + tableFolder.getName
-                val identifier =
-                  AbsoluteTableIdentifier.from(tablePath, dbName, tableFolder.getName)
-                val carbonTablePath = CarbonStorePath.getCarbonTablePath(identifier)
-                val tableStatusFile = carbonTablePath.getTableStatusFilePath
-                if (FileFactory.isFileExist(tableStatusFile, fileType)) {
-                  val segmentStatusManager = new SegmentStatusManager(identifier)
-                  val carbonLock = segmentStatusManager.getTableStatusLock
-                  try {
+        if (file.isDirectory) {
+          val tableFolders = file.listFiles()
+          tableFolders.foreach { tableFolder =>
+            if (tableFolder.isDirectory) {
+              val tablePath = databaseLocation +
+                              CarbonCommonConstants.FILE_SEPARATOR + tableFolder.getName
+              val identifier =
+                AbsoluteTableIdentifier.from(tablePath, dbName, tableFolder.getName)
+              val tableStatusFile =
+                CarbonTablePath.getTableStatusFilePath(tablePath)
+              if (FileFactory.isFileExist(tableStatusFile, fileType)) {
+                try {
                   val carbonTable = CarbonMetadata.getInstance
                     .getCarbonTable(identifier.getCarbonTableIdentifier.getTableUniqueName)
                   DataLoadingUtil.deleteLoadsAndUpdateMetadata(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 5e9f7fe..c8331f2 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
@@ -344,8 +344,8 @@ object DataLoadingUtil {
   def deleteLoadsAndUpdateMetadata(
       isForceDeletion: Boolean,
       carbonTable: CarbonTable): Unit = {
-    if (isLoadDeletionRequired(carbonTable.getMetaDataFilepath)) {
-      val details = SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath)
+    if (isLoadDeletionRequired(carbonTable.getMetadataPath)) {
+      val details = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
       val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
       val carbonTableStatusLock =
         CarbonLockFactory.getCarbonLockObj(
@@ -359,7 +359,7 @@ object DataLoadingUtil {
           absoluteTableIdentifier,
           isForceDeletion,
           details,
-          carbonTable.getMetaDataFilepath
+          carbonTable.getMetadataPath
         )
 
       var updationCompletionStaus = false
@@ -372,7 +372,7 @@ object DataLoadingUtil {
 
             // read latest table status again.
             val latestMetadata = SegmentStatusManager
-              .readLoadMetadata(carbonTable.getMetaDataFilepath)
+              .readLoadMetadata(carbonTable.getMetadataPath)
 
             // update the metadata details from old to new status.
             val latestStatus = CarbonLoaderUtil
@@ -397,7 +397,7 @@ object DataLoadingUtil {
         if (updationCompletionStaus) {
           DeleteLoadFolders
             .physicalFactAndMeasureMetadataDeletion(absoluteTableIdentifier,
-              carbonTable.getMetaDataFilepath, isForceDeletion)
+              carbonTable.getMetadataPath, isForceDeletion)
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 5f44e43..bbf345c 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
@@ -52,7 +52,7 @@ import org.apache.carbondata.core.reader.CarbonDictionaryReader
 import org.apache.carbondata.core.service.CarbonCommonFactory
 import org.apache.carbondata.core.statusmanager.SegmentStatus
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUtil}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.writer.CarbonDictionaryWriter
 import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.csvinput.{CSVInputFormat, StringArrayWritable}
@@ -308,7 +308,7 @@ object GlobalDictionaryUtil {
     }
     val primDimensions = primDimensionsBuffer.map { x => x }.toArray
     val dictDetail = CarbonSparkFactory.getDictionaryDetailService.
-      getDictionaryDetail(dictFolderPath, primDimensions, table, carbonLoadModel.getTablePath)
+      getDictionaryDetail(dictFolderPath, primDimensions, carbonLoadModel.getTablePath)
     val dictFilePaths = dictDetail.dictFilePaths
     val dictFileExists = dictDetail.dictFileExists
     val columnIdentifier = dictDetail.columnIdentifiers
@@ -398,10 +398,6 @@ object GlobalDictionaryUtil {
     }
   }
 
-  // Hack for spark2 integration
-  var updateTableMetadataFunc: (CarbonLoadModel, SQLContext, DictionaryLoadModel,
-    Array[CarbonDimension]) => Unit = _
-
   /**
    * check whether global dictionary have been generated successfully or not
    *
@@ -705,10 +701,7 @@ object GlobalDictionaryUtil {
     try {
       val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
       val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
-      // create dictionary folder if not exists
-      val tablePath = carbonLoadModel.getTablePath
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(tablePath, carbonTableIdentifier)
-      val dictfolderPath = carbonTablePath.getMetadataDirectoryPath
+      val dictfolderPath = CarbonTablePath.getMetadataPath(carbonLoadModel.getTablePath)
       // columns which need to generate global dictionary file
       val dimensions = carbonTable.getDimensionByTableName(
         carbonTable.getTableName).asScala.toArray
@@ -845,12 +838,11 @@ object GlobalDictionaryUtil {
    * This method will write dictionary file, sortindex file and dictionary meta for new dictionary
    * column with default value
    *
-   * @param carbonTablePath
    * @param columnSchema
    * @param absoluteTableIdentifier
    * @param defaultValue
    */
-  def loadDefaultDictionaryValueForNewColumn(carbonTablePath: CarbonTablePath,
+  def loadDefaultDictionaryValueForNewColumn(
       columnSchema: ColumnSchema,
       absoluteTableIdentifier: AbsoluteTableIdentifier,
       defaultValue: String): Unit = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 9a0098e..3b588df 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
@@ -29,7 +29,7 @@ import org.apache.spark.sql.util.CarbonException
 
 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, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.PartitionMapFileStore.PartitionMapper
 import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes, DecimalType}
 import org.apache.carbondata.core.metadata.encoder.Encoding
@@ -37,6 +37,7 @@ import org.apache.carbondata.core.metadata.schema._
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier, TableInfo, TableSchema}
 import org.apache.carbondata.core.metadata.schema.table.column.{ColumnSchema, ParentColumnTableRelation}
 import org.apache.carbondata.core.service.CarbonCommonFactory
+import org.apache.carbondata.core.service.impl.ColumnUniqueIdGenerator
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentUpdateStatusManager}
 import org.apache.carbondata.core.util.DataTypeUtil
 import org.apache.carbondata.core.util.path.CarbonTablePath
@@ -207,8 +208,7 @@ class AlterTableColumnSchemaGenerator(
     alterTableModel: AlterTableAddColumnsModel,
     dbName: String,
     tableInfo: TableInfo,
-    carbonTablePath: CarbonTablePath,
-    tableIdentifier: CarbonTableIdentifier,
+    tableIdentifier: AbsoluteTableIdentifier,
     sc: SparkContext) {
 
   val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
@@ -369,7 +369,7 @@ object TableNewProcessor {
       encoders.add(Encoding.DIRECT_DICTIONARY)
     }
     columnSchema.setEncodingList(encoders)
-    val colUniqueIdGenerator = CarbonCommonFactory.getColumnUniqueIdGenerator
+    val colUniqueIdGenerator = ColumnUniqueIdGenerator.getInstance
     val columnUniqueId = colUniqueIdGenerator.generateUniqueId(columnSchema)
     columnSchema.setColumnUniqueId(columnUniqueId)
     columnSchema.setColumnReferenceId(columnUniqueId)
@@ -433,7 +433,7 @@ class TableNewProcessor(cm: TableModel) {
       }
     }
     columnSchema.setEncodingList(encoders)
-    val colUniqueIdGenerator = CarbonCommonFactory.getColumnUniqueIdGenerator
+    val colUniqueIdGenerator = ColumnUniqueIdGenerator.getInstance
     val columnUniqueId = colUniqueIdGenerator.generateUniqueId(columnSchema)
     columnSchema.setColumnUniqueId(columnUniqueId)
     columnSchema.setColumnReferenceId(columnUniqueId)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 3c871db..1656efa 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
@@ -176,8 +176,6 @@ object PartitionUtils {
       getPartitionBlockList(identifier, segmentId, partitionIds, oldPartitionIds,
         partitionInfo, carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable).asScala
     val pathList: util.List[String] = new util.ArrayList[String]()
-    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)
@@ -190,8 +188,8 @@ object PartitionUtils {
         val batchNo = CarbonTablePath.DataFileUtil.getBatchNoFromTaskNo(taskNo)
         val taskId = CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskNo)
         val bucketNumber = CarbonTablePath.DataFileUtil.getBucketNo(path)
-        val indexFilePath = carbonTablePath.getCarbonIndexFilePath(
-          String.valueOf(taskId), segmentId, batchNo, String.valueOf(bucketNumber),
+        val indexFilePath = CarbonTablePath.getCarbonIndexFilePath(
+          tablePath, String.valueOf(taskId), segmentId, batchNo, String.valueOf(bucketNumber),
           timestamp, version)
         // indexFilePath could be duplicated when multiple data file related to one index file
         if (indexFilePath != null && !pathList.contains(indexFilePath)) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
index 5f8f389..adf33ff 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
@@ -27,7 +27,7 @@ import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.OperationContext
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CompactionType}
@@ -70,7 +70,7 @@ class AggregateDataMapCompactor(carbonLoadModel: CarbonLoadModel,
                     sqlContext.sparkSession, loadCommand.logicalPlan.get))
         loadCommand.processData(sqlContext.sparkSession)
         val newLoadMetaDataDetails = SegmentStatusManager.readLoadMetadata(
-          carbonTable.getMetaDataFilepath)
+          carbonTable.getMetadataPath)
         val updatedLoadMetaDataDetails = newLoadMetaDataDetails collect {
           case load if loadMetaDataDetails.contains(load) =>
             load.setMergedLoadName(mergedLoadName)
@@ -79,12 +79,9 @@ class AggregateDataMapCompactor(carbonLoadModel: CarbonLoadModel,
             load
           case other => other
         }
-        val carbonTablePath = CarbonStorePath
-          .getCarbonTablePath(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-            .getAbsoluteTableIdentifier)
-        SegmentStatusManager
-          .writeLoadDetailsIntoFile(carbonTablePath.getTableStatusFilePath,
-            updatedLoadMetaDataDetails)
+        SegmentStatusManager.writeLoadDetailsIntoFile(
+          CarbonTablePath.getTableStatusFilePath(carbonLoadModel.getTablePath),
+          updatedLoadMetaDataDetails)
         carbonLoadModel.setLoadMetadataDetails(updatedLoadMetaDataDetails.toList.asJava)
       } finally {
         // check if any other segments needs compaction on in case of MINOR_COMPACTION.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 655e5a0..f47c9bc 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
@@ -57,7 +57,7 @@ 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, CarbonUtil}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.FailureCauses
@@ -70,7 +70,7 @@ import org.apache.carbondata.processing.merger.{CarbonCompactionUtil, CarbonData
 import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
 import org.apache.carbondata.spark.{DataLoadResultImpl, PartitionFactory, _}
 import org.apache.carbondata.spark.load._
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, DataLoadingUtil, Util}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
 
 /**
  * This is the factory class which can create different RDD depends on user needs.
@@ -137,7 +137,7 @@ object CarbonDataRDDFactory {
       LOGGER.error("Not able to acquire the compaction lock for table " +
           s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
       CarbonCompactionUtil
-          .createCompactionRequiredFile(carbonTable.getMetaDataFilepath, compactionType)
+          .createCompactionRequiredFile(carbonTable.getMetadataPath, compactionType)
       // throw exception only in case of DDL trigger.
       if (compactionModel.isDDLTrigger) {
         CarbonException.analysisException(
@@ -205,7 +205,7 @@ object CarbonDataRDDFactory {
                   s"${ tableForCompaction.getDatabaseName }." +
                   s"${ tableForCompaction.getTableName}")
               val table: CarbonTable = tableForCompaction
-              val metadataPath = table.getMetaDataFilepath
+              val metadataPath = table.getMetadataPath
               val compactionType = CarbonCompactionUtil.determineCompactionType(metadataPath)
 
               val newCarbonLoadModel = prepareCarbonLoadModel(table)
@@ -596,15 +596,13 @@ object CarbonDataRDDFactory {
         (row.get(row.size - 1).toString, Row(row.toSeq.slice(0, row.size - 1): _*)))
 
       val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
-        carbonTable.getMetaDataFilepath)
+        carbonTable.getMetadataPath)
         .filter(lmd => lmd.getSegmentStatus.equals(SegmentStatus.LOAD_PARTIAL_SUCCESS) ||
                        lmd.getSegmentStatus.equals(SegmentStatus.SUCCESS))
       val segmentIds = loadMetadataDetails.map(_.getLoadName)
       val segmentIdIndex = segmentIds.zipWithIndex.toMap
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonLoadModel.getTablePath,
-        carbonTable.getCarbonTableIdentifier)
       val segmentId2maxTaskNo = segmentIds.map { segId =>
-        (segId, CarbonUpdateUtil.getLatestTaskIdForSegment(segId, carbonTablePath))
+        (segId, CarbonUpdateUtil.getLatestTaskIdForSegment(segId, carbonLoadModel.getTablePath))
       }.toMap
 
       class SegmentPartitioner(segIdIndex: Map[String, Int], parallelism: Int)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
index a0c8f65..ddc8586 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
@@ -156,7 +156,7 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
     }
     val carbonMergerMapping = CarbonMergerMapping(
       tablePath,
-      carbonTable.getMetaDataFilepath,
+      carbonTable.getMetadataPath,
       mergedLoadName,
       databaseName,
       factTableName,
@@ -169,7 +169,7 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
       partitionMapper)
     carbonLoadModel.setTablePath(carbonMergerMapping.hdfsStoreLocation)
     carbonLoadModel.setLoadMetadataDetails(
-      SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath).toList.asJava)
+      SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath).toList.asJava)
     // trigger event for compaction
     val alterTableCompactionPreEvent: AlterTableCompactionPreEvent =
       AlterTableCompactionPreEvent(sqlContext.sparkSession,
@@ -240,10 +240,10 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
         ((compactionType == CompactionType.IUD_UPDDEL_DELTA) &&
          CarbonDataMergerUtil
            .updateLoadMetadataIUDUpdateDeltaMergeStatus(loadsToMerge,
-             carbonTable.getMetaDataFilepath,
+             carbonTable.getMetadataPath,
              carbonLoadModel)) ||
         CarbonDataMergerUtil
-          .updateLoadMetadataWithMergeStatus(loadsToMerge, carbonTable.getMetaDataFilepath,
+          .updateLoadMetadataWithMergeStatus(loadsToMerge, carbonTable.getMetadataPath,
             mergedLoadNumber, carbonLoadModel, compactionType)
 
       if (!statusFileUpdation) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
index fb0f9fe..febb83e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
@@ -37,8 +37,9 @@ import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{AlterTableCompactionPostEvent, AlterTableCompactionPreEvent, AlterTableCompactionPreStatusUpdateEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.loading.events.LoadEvents.LoadMetadataEvent
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
@@ -237,7 +238,7 @@ case class CarbonAlterTableCompactionCommand(
               readFileFooterFromCarbonDataFile = true)
 
             val carbonMergerMapping = CarbonMergerMapping(carbonTable.getTablePath,
-              carbonTable.getMetaDataFilepath,
+              carbonTable.getMetadataPath,
               "",
               carbonTable.getDatabaseName,
               carbonTable.getTableName,
@@ -312,9 +313,10 @@ case class CarbonAlterTableCompactionCommand(
           true)(sparkSession,
           sparkSession.sessionState.catalog.asInstanceOf[CarbonSessionCatalog])
         // 5. remove checkpoint
-        val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-        FileFactory.deleteAllFilesOfDir(new File(tablePath.getStreamingCheckpointDir))
-        FileFactory.deleteAllFilesOfDir(new File(tablePath.getStreamingLogDir))
+        FileFactory.deleteAllFilesOfDir(
+          new File(CarbonTablePath.getStreamingCheckpointDir(carbonTable.getTablePath)))
+        FileFactory.deleteAllFilesOfDir(
+          new File(CarbonTablePath.getStreamingLogDir(carbonTable.getTablePath)))
       } else {
         val msg = "Failed to close streaming table, because streaming is locked for table " +
                   carbonTable.getDatabaseName() + "." + carbonTable.getTableName()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index 226a625..c7b59d4 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -59,7 +59,7 @@ import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
 import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.util.path.{CarbonStorePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.events.exception.PreEventException
 import org.apache.carbondata.hadoop.util.ObjectSerializationUtil
@@ -67,8 +67,8 @@ import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.TableProcessingOperations
 import org.apache.carbondata.processing.loading.csvinput.{CSVInputFormat, StringArrayWritable}
 import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadMetadataEvent, LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
-import org.apache.carbondata.processing.loading.exception.{NoRetryException}
-import org.apache.carbondata.processing.loading.model.{CarbonLoadModel}
+import org.apache.carbondata.processing.loading.exception.NoRetryException
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
 import org.apache.carbondata.spark.dictionary.provider.SecureDictionaryServiceProvider
 import org.apache.carbondata.spark.dictionary.server.SecureDictionaryServer
@@ -217,8 +217,7 @@ case class CarbonLoadDataCommand(
           carbonLoadModel.setUseOnePass(false)
         }
         // Create table and metadata folders if not exist
-        val carbonTablePath = CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier)
-        val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
+        val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)
         val fileType = FileFactory.getFileType(metadataDirectoryPath)
         if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
           FileFactory.mkdirs(metadataDirectoryPath, fileType)
@@ -309,9 +308,7 @@ case class CarbonLoadDataCommand(
     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
     val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
       .getCarbonTableIdentifier
-    val carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(carbonLoadModel.getTablePath, carbonTableIdentifier)
-    val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
+    val dictFolderPath = CarbonTablePath.getMetadataPath(carbonLoadModel.getTablePath)
     val dimensions = carbonTable.getDimensionByTableName(
       carbonTable.getTableName).asScala.toArray
     val colDictFilePath = carbonLoadModel.getColDictFilePath
@@ -880,4 +877,5 @@ case class CarbonLoadDataCommand(
     val dataFrameWithTupleId = dataFrame.get.select(fieldWithTupleId: _*)
     (dataFrameWithTupleId)
   }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
index f8f215f..1e5885e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
@@ -45,7 +45,7 @@ case class CarbonShowLoadsCommand(
     val carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
     CarbonStore.showSegments(
       limit,
-      carbonTable.getMetaDataFilepath
+      carbonTable.getMetadataPath
     )
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
index 2983ea4..90ff3b4 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
@@ -35,7 +35,7 @@ import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
 import org.apache.carbondata.hadoop.util.SchemaReader
 
@@ -62,19 +62,18 @@ case class RefreshCarbonTableCommand(
     // 2.2.1 validate that all the aggregate tables are copied at the store location.
     // 2.2.2 Register the aggregate tables
     val tablePath = CarbonEnv.getTablePath(databaseNameOp, tableName)(sparkSession)
-    val absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, databaseName, tableName)
+    val identifier = AbsoluteTableIdentifier.from(tablePath, databaseName, tableName)
     // 2.1 check if the table already register with hive then ignore and continue with the next
     // schema
     if (!sparkSession.sessionState.catalog.listTables(databaseName)
       .exists(_.table.equalsIgnoreCase(tableName))) {
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
       // check the existence of the schema file to know its a carbon table
-      val schemaFilePath = carbonTablePath.getSchemaFilePath
+      val schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath)
       // if schema file does not exist then the table will either non carbon table or stale
       // carbon table
       if (FileFactory.isFileExist(schemaFilePath, FileFactory.getFileType(schemaFilePath))) {
         // read TableInfo
-        val tableInfo = SchemaReader.getTableInfo(absoluteTableIdentifier)
+        val tableInfo = SchemaReader.getTableInfo(identifier)
         // 2.2 register the table with the hive check if the table being registered has
         // aggregate table then do the below steps
         // 2.2.1 validate that all the aggregate tables are copied at the store location.
@@ -98,7 +97,7 @@ case class RefreshCarbonTableCommand(
         // Register partitions to hive metastore in case of hive partitioning carbon table
         if (tableInfo.getFactTable.getPartitionInfo != null &&
             tableInfo.getFactTable.getPartitionInfo.getPartitionType == PartitionType.NATIVE_HIVE) {
-          registerAllPartitionsToHive(absoluteTableIdentifier, sparkSession)
+          registerAllPartitionsToHive(identifier, sparkSession)
         }
       } else {
         LOGGER.audit(
@@ -177,9 +176,7 @@ case class RefreshCarbonTableCommand(
     dataMapSchemaList.asScala.foreach(dataMap => {
       val tableName = dataMap.getChildSchema.getTableName
       val tablePath = CarbonEnv.getTablePath(Some(dbName), tableName)(sparkSession)
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(tablePath,
-        new CarbonTableIdentifier(dbName, tableName, dataMap.getChildSchema.getTableId))
-      val schemaFilePath = carbonTablePath.getSchemaFilePath
+      val schemaFilePath = CarbonTablePath.getSchemaFilePath(tablePath)
       try {
         fileExist = FileFactory.isFileExist(schemaFilePath, FileFactory.getFileType(schemaFilePath))
       } catch {
@@ -190,7 +187,7 @@ case class RefreshCarbonTableCommand(
         return fileExist;
       }
     })
-    return true
+    true
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 a8efb84..58456b7 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
@@ -40,7 +40,7 @@ import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, DeleteDeltaBlockDetails, SegmentUpdateDetails, TupleIdEnum}
 import org.apache.carbondata.core.mutate.data.RowCountDetailsVO
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentUpdateStatusManager}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
 import org.apache.carbondata.processing.exception.MultipleMatchingException
@@ -67,8 +67,7 @@ object DeleteExecution {
     val database = CarbonEnv.getDatabaseName(databaseNameOp)(sparkSession)
     val carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
     val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
-    val factPath = carbonTablePath.getFactDir
+    val factPath = CarbonTablePath.getFactDir(carbonTable.getTablePath)
     var segmentsTobeDeleted = Seq.empty[String]
 
     val deleteRdd = if (isUpdateOperation) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
index 114c25d..b53c609 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
@@ -39,7 +39,7 @@ import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.loading.TableProcessingOperations
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.spark.partition.DropPartitionCallable
@@ -69,8 +69,8 @@ case class CarbonAlterTableDropPartitionCommand(
       LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
       sys.error(s"Alter table failed. table not found: $dbName.$tableName")
     }
-    val table = relation.carbonTable
-    val partitionInfo = table.getPartitionInfo(tableName)
+    val carbonTable = relation.carbonTable
+    val partitionInfo = carbonTable.getPartitionInfo(tableName)
     if (partitionInfo == null) {
       sys.error(s"Table $tableName is not a partition table.")
     }
@@ -95,11 +95,9 @@ case class CarbonAlterTableDropPartitionCommand(
         sys.error(s"Dropping range interval partition isn't support yet!")
     }
     partitionInfo.dropPartition(partitionIndex)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier)
-    val schemaFilePath = carbonTablePath.getSchemaFilePath
-    // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
 
+    // read TableInfo
+    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)(sparkSession)
     val schemaConverter = new ThriftWrapperSchemaConverterImpl()
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
       dbName, tableName, tablePath)
@@ -112,11 +110,11 @@ case class CarbonAlterTableDropPartitionCommand(
     thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
       .setTime_stamp(System.currentTimeMillis)
     carbonMetaStore.updateTableSchemaForAlter(
-      table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-      table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
       thriftTable,
       null,
-      table.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+      carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
     // update the schema modified time
     carbonMetaStore.updateAndTouchSchemasUpdatedTime()
     // sparkSession.catalog.refreshTable(tableName)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
index bafc96a..84779cc 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
@@ -41,7 +41,7 @@ import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.loading.TableProcessingOperations
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.spark.partition.SplitPartitionCallable
@@ -72,8 +72,8 @@ case class CarbonAlterTableSplitPartitionCommand(
       LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
       sys.error(s"Alter table failed. table not found: $dbName.$tableName")
     }
-    val table = relation.carbonTable
-    val partitionInfo = table.getPartitionInfo(tableName)
+    val carbonTable = relation.carbonTable
+    val partitionInfo = carbonTable.getPartitionInfo(tableName)
     val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
     // keep a copy of partitionIdList before update partitionInfo.
     // will be used in partition data scan
@@ -88,10 +88,8 @@ case class CarbonAlterTableSplitPartitionCommand(
 
     updatePartitionInfo(partitionInfo, partitionIds)
 
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier)
-    val schemaFilePath = carbonTablePath.getSchemaFilePath
     // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)(sparkSession)
     val schemaConverter = new ThriftWrapperSchemaConverterImpl()
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
       dbName, tableName, tablePath)
@@ -101,12 +99,12 @@ case class CarbonAlterTableSplitPartitionCommand(
     wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
     val thriftTable =
       schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-    carbonMetaStore
-      .updateTableSchemaForAlter(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-        table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-        thriftTable,
-        null,
-        table.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+    carbonMetaStore.updateTableSchemaForAlter(
+      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+      thriftTable,
+      null,
+      carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
     // update the schema modified time
     carbonMetaStore.updateAndTouchSchemasUpdatedTime()
     Seq.empty

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
index dbbf90c..fed4235 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
@@ -183,7 +183,7 @@ case class CreatePreAggregateTableCommand(
     // need to fire load for pre-aggregate table. Therefore reading the load details for PARENT
     // table.
     DataLoadingUtil.deleteLoadsAndUpdateMetadata(isForceDeletion = false, parentTable)
-    val loadAvailable = SegmentStatusManager.readLoadMetadata(parentTable.getMetaDataFilepath)
+    val loadAvailable = SegmentStatusManager.readLoadMetadata(parentTable.getMetadataPath)
     if (loadAvailable.exists(load => load.getSegmentStatus == SegmentStatus.INSERT_IN_PROGRESS ||
       load.getSegmentStatus == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)) {
       throw new UnsupportedOperationException(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 dac5d5e..feef7a1 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
@@ -16,11 +16,12 @@
  */
 package org.apache.spark.sql.execution.command.preaaggregate
 
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 import scala.collection.JavaConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 
-import org.apache.spark.sql._
-import org.apache.spark.sql.CarbonExpressions.{CarbonSubqueryAlias => SubqueryAlias, MatchCastExpression}
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, CarbonSession, SparkSession, _}
+import org.apache.spark.sql.CarbonExpressions.{CarbonSubqueryAlias => SubqueryAlias}
+import org.apache.spark.sql.CarbonExpressions.MatchCastExpression
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedFunction, UnresolvedRelation}
 import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, AttributeSeq, Cast, Expression, ExprId, NamedExpression, ScalaUDF}
@@ -40,7 +41,6 @@ import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverte
 import org.apache.carbondata.core.metadata.schema.table.{AggregationDataMapSchema, CarbonTable, DataMapSchema, TableSchema}
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.format.TableInfo
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.CommonUtil
@@ -423,9 +423,7 @@ 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.getAbsoluteTableIdentifier)
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
       val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
         thriftTableInfo,
@@ -528,8 +526,7 @@ object PreAggregateUtil {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
     carbonTable.getTableLastUpdatedTime
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
     if (thriftTable.dataMapSchemas.size > numberOfChildSchema) {
       metastore.revertTableSchemaForPreAggCreationFailure(
         carbonTable.getAbsoluteTableIdentifier, thriftTable)(sparkSession)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 f3f01bb..07917d0 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
@@ -28,7 +28,6 @@ import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.events.{AlterTableAddColumnPostEvent, AlterTableAddColumnPreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.TableInfo
 import org.apache.carbondata.spark.rdd.{AlterTableAddColumnRDD, AlterTableDropColumnRDD}
@@ -64,9 +63,7 @@ private[sql] case class CarbonAlterTableAddColumnCommand(
       OperationListenerBus.getInstance().fireEvent(alterTableAddColumnListener, operationContext)
       // get the latest carbon table and check for column existence
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
       val wrapperTableInfo = schemaConverter
         .fromExternalToWrapperTableInfo(thriftTableInfo,
@@ -76,8 +73,7 @@ private[sql] case class CarbonAlterTableAddColumnCommand(
       newCols = new AlterTableColumnSchemaGenerator(alterTableAddColumnsModel,
         dbName,
         wrapperTableInfo,
-        carbonTablePath,
-        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getAbsoluteTableIdentifier,
         sparkSession.sparkContext).process
       // generate dictionary files for the newly added columns
       new AlterTableAddColumnRDD(sparkSession.sparkContext,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 9bea935..fa8003e 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
@@ -27,7 +27,7 @@ import org.apache.spark.util.AlterTableUtil
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{AlterTableDataTypeChangePostEvent, AlterTableDataTypeChangePreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.{ColumnSchema, SchemaEvolutionEntry, TableInfo}
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataTypeConverterUtil}
@@ -74,9 +74,7 @@ private[sql] case class CarbonAlterTableDataTypeChangeCommand(
         sys.error(s"Invalid Column: $columnName")
       }
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
       // maintain the added column for schema evolution history
       var addColumnSchema: ColumnSchema = null
       var deletedColumnSchema: ColumnSchema = null

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 0319d9e..d848eb5 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
@@ -29,7 +29,7 @@ import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{AlterTableDropColumnPostEvent, AlterTableDropColumnPreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.spark.rdd.AlterTableDropColumnRDD
@@ -98,10 +98,8 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
       OperationListenerBus.getInstance().fireEvent(alterTableDropColumnPreEvent, operationContext)
 
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
       val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+        metastore.getThriftTableInfo(carbonTable)(sparkSession)
       // maintain the deleted columns for schema evolution history
       var deletedColumnSchema = ListBuffer[org.apache.carbondata.format.ColumnSchema]()
       val columnSchemaList = tableInfo.fact_table.table_columns.asScala

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 dd34f08..7a56dbf 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
@@ -34,7 +34,7 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 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.events.{AlterTableRenamePostEvent, AlterTableRenamePreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
@@ -98,8 +98,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
       val oldTableIdentifier = carbonTable.getAbsoluteTableIdentifier
       DataMapStoreManager.getInstance().clearDataMaps(oldTableIdentifier)
       // get the latest carbon table and check for column existence
-      val oldTablePath = CarbonStorePath.getCarbonTablePath(oldTableIdentifier)
-      val tableMetadataFile = oldTablePath.getPath
+      val tableMetadataFile = oldTableIdentifier.getTablePath
       val operationContext = new OperationContext
       // TODO: Pass new Table Path in pre-event.
       val alterTableRenamePreEvent: AlterTableRenamePreEvent = AlterTableRenamePreEvent(
@@ -109,7 +108,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
         sparkSession)
       OperationListenerBus.getInstance().fireEvent(alterTableRenamePreEvent, operationContext)
       val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(oldTablePath)(sparkSession)
+        metastore.getThriftTableInfo(carbonTable)(sparkSession)
       val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
       schemaEvolutionEntry.setTableName(newTableName)
       timeStamp = System.currentTimeMillis()
@@ -118,7 +117,8 @@ private[sql] case class CarbonAlterTableRenameCommand(
       val fileType = FileFactory.getFileType(tableMetadataFile)
       val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
         newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
-      var newTablePath = CarbonUtil.getNewTablePath(oldTablePath, newTableIdentifier.getTableName)
+      var newTablePath = CarbonTablePath.getNewTablePath(
+        oldTableIdentifier.getTablePath, newTableIdentifier.getTableName)
       metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
       val hiveClient = sparkSession.sessionState.catalog.asInstanceOf[CarbonSessionCatalog]
         .getClient()
@@ -133,9 +133,9 @@ private[sql] case class CarbonAlterTableRenameCommand(
       // 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(oldTablePath.getPath, fileType)
-          .renameForce(oldTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
-                       newTableName)
+        val rename = FileFactory.getCarbonFile(oldTableIdentifier.getTablePath, fileType)
+          .renameForce(
+            CarbonTablePath.getNewTablePath(oldTableIdentifier.getTablePath, newTableName))
         if (!rename) {
           renameBadRecords(newTableName, oldTableName, oldDatabaseName)
           sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
@@ -163,13 +163,11 @@ private[sql] case class CarbonAlterTableRenameCommand(
       case e: Exception =>
         LOGGER.error(e, "Rename table failed: " + e.getMessage)
         if (carbonTable != null) {
-          AlterTableUtil
-            .revertRenameTableChanges(oldTableIdentifier,
-              newTableName,
-              carbonTable.getTablePath,
-              carbonTable.getCarbonTableIdentifier.getTableId,
-              timeStamp)(
-              sparkSession)
+          AlterTableUtil.revertRenameTableChanges(
+            newTableName,
+            carbonTable,
+            timeStamp)(
+            sparkSession)
           renameBadRecords(newTableName, oldTableName, oldDatabaseName)
         }
         sys.error(s"Alter table rename table operation failed: ${e.getMessage}")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 b44dc7e..fd09e48 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
@@ -46,7 +46,8 @@ import org.apache.carbondata.core.metadata.schema
 import org.apache.carbondata.core.metadata.schema.table
 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.util.path.CarbonTablePath
+import org.apache.carbondata.core.util.path.CarbonTablePath.getNewTablePath
 import org.apache.carbondata.core.writer.ThriftWriter
 import org.apache.carbondata.events.{LookupRelationPostEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
@@ -209,11 +210,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
     val dbName = identifier.getCarbonTableIdentifier.getDatabaseName
     val tableName = identifier.getCarbonTableIdentifier.getTableName
     val tablePath = identifier.getTablePath
-    val carbonTableIdentifier = new CarbonTableIdentifier(dbName.toLowerCase(),
-      tableName.toLowerCase(), UUID.randomUUID().toString)
-    val carbonTablePath =
-      CarbonStorePath.getCarbonTablePath(tablePath, carbonTableIdentifier)
-    val tableMetadataFile = carbonTablePath.getSchemaFilePath
+    val tableMetadataFile = CarbonTablePath.getSchemaFilePath(tablePath)
     val fileType = FileFactory.getFileType(tableMetadataFile)
     if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
       val tableUniqueName = CarbonTable.buildUniqueName(dbName, tableName)
@@ -240,13 +237,13 @@ class CarbonFileMetastore extends CarbonMetaStore {
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
       schemaEvolutionEntry: SchemaEvolutionEntry,
       tablePath: String) (sparkSession: SparkSession): String = {
-    val absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, oldTableIdentifier)
+    val identifier = AbsoluteTableIdentifier.from(tablePath, oldTableIdentifier)
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
     if (schemaEvolutionEntry != null) {
       thriftTableInfo.fact_table.schema_evolution.schema_evolution_history.add(schemaEvolutionEntry)
     }
-    val oldTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
-    val newTablePath = CarbonUtil.getNewTablePath(oldTablePath, newTableIdentifier.getTableName)
+    val newTablePath = CarbonTablePath.getNewTablePath(
+      identifier.getTablePath, newTableIdentifier.getTableName)
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
       thriftTableInfo,
       newTableIdentifier.getDatabaseName,
@@ -341,8 +338,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
   private def createSchemaThriftFile(
       identifier: AbsoluteTableIdentifier,
       thriftTableInfo: TableInfo): String = {
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(identifier)
-    val schemaFilePath = carbonTablePath.getSchemaFilePath
+    val schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath)
     val schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath)
     val fileType = FileFactory.getFileType(schemaMetadataPath)
     if (!FileFactory.isFileExist(schemaMetadataPath, fileType)) {
@@ -356,7 +352,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
     thriftWriter.write(thriftTableInfo)
     thriftWriter.close()
     updateSchemasUpdatedTime(touchSchemaFileSystemTime())
-    carbonTablePath.getPath
+    identifier.getTablePath
   }
 
   protected def addTableCache(
@@ -431,8 +427,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
     (sparkSession: SparkSession) {
     val dbName = absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName
     val tableName = absoluteTableIdentifier.getCarbonTableIdentifier.getTableName
-    val metadataFilePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
-      .getMetadataDirectoryPath
+    val metadataFilePath = CarbonTablePath.getMetadataPath(absoluteTableIdentifier.getTablePath)
     val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName, tableName)
     if (null != carbonTable) {
       // clear driver B-tree and dictionary cache
@@ -528,9 +523,9 @@ class CarbonFileMetastore extends CarbonMetaStore {
   override def listAllTables(sparkSession: SparkSession): Seq[CarbonTable] =
     metadata.carbonTables
 
-  override def getThriftTableInfo(tablePath: CarbonTablePath)
+  override def getThriftTableInfo(carbonTable: CarbonTable)
     (sparkSession: SparkSession): TableInfo = {
-    val tableMetadataFile = tablePath.getSchemaFilePath
+    val tableMetadataFile = CarbonTablePath.getSchemaFilePath(carbonTable.getTablePath)
     CarbonUtil.readSchemaFile(tableMetadataFile)
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 759471b..44f731e 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
@@ -28,7 +28,7 @@ import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetad
 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, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.format
 import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.spark.util.CarbonSparkUtil
@@ -96,12 +96,8 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
     Seq()
   }
 
-  override def getThriftTableInfo(tablePath: CarbonTablePath)
+  override def getThriftTableInfo(carbonTable: CarbonTable)
     (sparkSession: SparkSession): format.TableInfo = {
-    val identifier = tablePath.getCarbonTableIdentifier
-    val relation = lookupRelation(TableIdentifier(identifier.getTableName,
-      Some(identifier.getDatabaseName)))(sparkSession).asInstanceOf[CarbonRelation]
-    val carbonTable = relation.metaData.carbonTable
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
     schemaConverter.fromWrapperToExternalTableInfo(carbonTable.getTableInfo,
       carbonTable.getDatabaseName,
@@ -148,7 +144,8 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
       carbonTablePath: String)(sparkSession: SparkSession): String = {
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
-    updateHiveMetaStoreForAlter(newTableIdentifier,
+    updateHiveMetaStoreForAlter(
+      newTableIdentifier,
       oldTableIdentifier,
       thriftTableInfo,
       carbonTablePath,
@@ -163,7 +160,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
       sparkSession: SparkSession,
       schemaConverter: ThriftWrapperSchemaConverterImpl) = {
     val newTablePath =
-      CarbonUtil.getNewTablePath(new Path(oldTablePath), newTableIdentifier.getTableName)
+      CarbonTablePath.getNewTablePath(oldTablePath, newTableIdentifier.getTableName)
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
       thriftTableInfo,
       newTableIdentifier.getDatabaseName,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 93c7c09..0645040 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
@@ -143,7 +143,7 @@ trait CarbonMetaStore {
 
   def listAllTables(sparkSession: SparkSession): Seq[CarbonTable]
 
-  def getThriftTableInfo(tablePath: CarbonTablePath)(sparkSession: SparkSession): TableInfo
+  def getThriftTableInfo(carbonTable: CarbonTable)(sparkSession: SparkSession): TableInfo
 
   def getTableFromMetadataCache(database: String, tableName: String): Option[CarbonTable]
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
index b8608f4..a722838 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
@@ -34,7 +34,7 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, CarbonDimension}
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 /**
  * Represents logical plan for one carbon table
@@ -209,9 +209,7 @@ case class CarbonRelation(
         .getValidAndInvalidSegments.getValidSegments.isEmpty) {
         sizeInBytesLocalValue = 0L
       } else {
-        val tablePath = CarbonStorePath.getCarbonTablePath(
-          carbonTable.getTablePath,
-          carbonTable.getCarbonTableIdentifier).getPath
+        val tablePath = carbonTable.getTablePath
         val fileType = FileFactory.getFileType(tablePath)
         if (FileFactory.isFileExist(tablePath, fileType)) {
           // get the valid segments

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 8ebd5a9..bc36e9c 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
@@ -36,7 +36,8 @@ import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTable
 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.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.util.path.CarbonTablePath.getNewTablePath
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
@@ -176,41 +177,28 @@ object AlterTableUtil {
 
   /**
    * This method reverts the changes to the schema if the rename table command fails.
-   *
-   * @param oldTableIdentifier
-   * @param newTableName
-   * @param timeStamp
-   * @param sparkSession
    */
-  def revertRenameTableChanges(oldTableIdentifier: TableIdentifier,
+  def revertRenameTableChanges(
       newTableName: String,
-      tablePath: String,
-      tableId: String,
+      oldCarbonTable: CarbonTable,
       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 tablePath = oldCarbonTable.getTablePath
+    val tableId = oldCarbonTable.getCarbonTableIdentifier.getTableId
+    val oldCarbonTableIdentifier = oldCarbonTable.getCarbonTableIdentifier
+    val database = oldCarbonTable.getDatabaseName
     val newCarbonTableIdentifier = new CarbonTableIdentifier(database, newTableName, tableId)
-    val newTablePath = CarbonUtil.getNewTablePath(new Path(tablePath), newTableName)
+    val newTablePath = CarbonTablePath.getNewTablePath(tablePath, newTableName)
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val fileType = FileFactory.getFileType(tablePath)
     if (FileFactory.isFileExist(tablePath, fileType)) {
-      val tableInfo = if (metastore.isReadFromHiveMetaStore) {
-        // In case of hive metastore we first update the carbonschema inside old table only.
-        metastore.getThriftTableInfo(CarbonStorePath.getCarbonTablePath(tablePath,
-          new CarbonTableIdentifier(database, oldTableIdentifier.table, tableId)))(sparkSession)
-      } else {
-        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      }
+      val tableInfo = metastore.getThriftTableInfo(oldCarbonTable)(sparkSession)
       val evolutionEntryList = tableInfo.fact_table.schema_evolution.schema_evolution_history
       val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
       if (updatedTime == timeStamp) {
-        LOGGER.error(s"Reverting changes for $database.${ oldTableIdentifier.table }")
-        FileFactory.getCarbonFile(carbonTablePath.getPath, fileType)
-          .renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
-                       oldTableIdentifier.table)
+        LOGGER.error(s"Reverting changes for $database.${oldCarbonTable.getTableName}")
+        FileFactory.getCarbonFile(tablePath, fileType)
+          .renameForce(CarbonTablePath.getNewTablePath(tablePath, oldCarbonTable.getTableName))
         val absoluteTableIdentifier = AbsoluteTableIdentifier.from(
           newTablePath,
           newCarbonTableIdentifier)
@@ -233,9 +221,7 @@ object AlterTableUtil {
     (sparkSession: SparkSession): Unit = {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
-      carbonTable.getCarbonTableIdentifier)
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
     val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
     if (updatedTime == timeStamp) {
@@ -260,9 +246,7 @@ object AlterTableUtil {
     (sparkSession: SparkSession): Unit = {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
-      carbonTable.getCarbonTableIdentifier)
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
     val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
     if (updatedTime == timeStamp) {
@@ -293,9 +277,7 @@ object AlterTableUtil {
     (sparkSession: SparkSession): Unit = {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
-      carbonTable.getCarbonTableIdentifier)
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
     val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
     if (updatedTime == timeStamp) {
@@ -344,9 +326,7 @@ object AlterTableUtil {
       carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
       // get the latest carbon table
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
-        carbonTable.getCarbonTableIdentifier)
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
       val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
         thriftTableInfo,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 aadee81..0bdef8a 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
@@ -856,9 +856,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
   }
 
   def getDataFiles(carbonTable: CarbonTable, segmentId: String): Array[CarbonFile] = {
-    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
-      carbonTable.getTablePath)
-    val segmentDir = tablePath.getCarbonDataDirectoryPath(segmentId)
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {
       override def accept(file: CarbonFile): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 56c5747..71c5477 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
@@ -24,7 +24,7 @@ import org.scalatest.BeforeAndAfterAll
 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.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.processing.util.TableOptionConstant
 
@@ -65,9 +65,7 @@ class AllDictionaryTestCase extends Spark2QueryTest with BeforeAndAfterAll {
     carbonLoadModel.setCsvHeaderColumns(
       CommonUtil.getCsvHeaderColumns(carbonLoadModel, FileFactory.getConfiguration))
     // Create table and metadata folders if not exist
-    val carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(table.getTablePath, table.getCarbonTableIdentifier)
-    val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
+    val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)
     val fileType = FileFactory.getFileType(metadataDirectoryPath)
     if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
       FileFactory.mkdirs(metadataDirectoryPath, fileType)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 f9519f8..a465251 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
@@ -23,7 +23,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.{AbsoluteTableIdentifier, CarbonTableIdentifier}
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 a1b39d8..c0e1781 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
@@ -26,7 +26,7 @@ import org.scalatest.BeforeAndAfterAll
 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.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.processing.util.TableOptionConstant
@@ -179,9 +179,7 @@ class ExternalColumnDictionaryTestCase extends Spark2QueryTest with BeforeAndAft
       CommonUtil.getCsvHeaderColumns(carbonLoadModel, FileFactory.getConfiguration))
     carbonLoadModel.setMaxColumns("100")
     // Create table and metadata folders if not exist
-    val carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(table.getTablePath, table.getCarbonTableIdentifier)
-    val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
+    val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)
     val fileType = FileFactory.getFileType(metadataDirectoryPath)
     if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
       FileFactory.mkdirs(metadataDirectoryPath, fileType)


[4/4] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

Posted by ja...@apache.org.
[CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

Refactory CarbonTablePath:

1.Remove CarbonStorePath and use CarbonTablePath only.
2.Make CarbonTablePath an utility without object creation, it can avoid creating object before using it, thus code is cleaner and GC is less.

This closes #1768


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

Branch: refs/heads/carbonstore
Commit: c3e99681bcd397ed33bc90e8d73b1fd33e0e60f7
Parents: 15b4e19
Author: Jacky Li <ja...@qq.com>
Authored: Wed Jan 31 16:14:27 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Thu Feb 1 15:33:19 2018 +0800

----------------------------------------------------------------------
 .../DictionaryColumnUniqueIdentifier.java       |  29 +-
 .../dictionary/ManageDictionaryAndBTree.java    |  13 +-
 .../core/metadata/AbsoluteTableIdentifier.java  |   4 +-
 .../core/metadata/PartitionMapFileStore.java    |  15 +-
 .../core/metadata/schema/table/CarbonTable.java |  11 +-
 .../core/mutate/CarbonUpdateUtil.java           |  59 ++--
 .../executor/impl/AbstractQueryExecutor.java    |   8 +-
 .../core/scan/executor/util/QueryUtil.java      |   7 +-
 .../core/service/CarbonCommonFactory.java       |  16 -
 .../carbondata/core/service/PathService.java    |  35 ---
 .../core/service/impl/PathFactory.java          |  50 ----
 .../statusmanager/SegmentStatusManager.java     |  28 +-
 .../SegmentUpdateStatusManager.java             |  72 ++---
 .../apache/carbondata/core/util/CarbonUtil.java |  61 ++--
 .../util/path/CarbonSharedDictionaryPath.java   |  71 -----
 .../core/util/path/CarbonStorePath.java         |  71 -----
 .../core/util/path/CarbonTablePath.java         | 296 ++++++-------------
 .../dictionary/AbstractDictionaryCacheTest.java |  11 +-
 .../dictionary/ForwardDictionaryCacheTest.java  |   6 +-
 .../dictionary/ReverseDictionaryCacheTest.java  |   6 +-
 .../reader/CarbonDictionaryReaderImplTest.java  |   8 -
 .../CarbonFormatDirectoryStructureTest.java     |  18 +-
 .../path/CarbonFormatSharedDictionaryTest.java  |  44 ---
 .../writer/CarbonDictionaryWriterImplTest.java  |  19 +-
 .../CarbonBatchSparkStreamingExample.scala      |   9 +-
 .../CarbonStructuredStreamingExample.scala      |  11 +-
 .../hadoop/api/CarbonTableInputFormat.java      |   8 +-
 .../streaming/CarbonStreamRecordWriter.java     |   6 +-
 .../carbondata/hadoop/util/SchemaReader.java    |  18 +-
 .../hadoop/test/util/StoreCreator.java          |  16 +-
 .../presto/CarbondataRecordSetProvider.java     |   7 +-
 .../presto/impl/CarbonTableCacheModel.java      |  13 +-
 .../presto/impl/CarbonTableReader.java          |  49 ++-
 .../presto/util/CarbonDataStoreCreator.scala    |  10 +-
 .../sdv/generated/MergeIndexTestCase.scala      |   2 +-
 .../dataload/TestLoadDataGeneral.scala          |  11 +-
 .../InsertIntoCarbonTableTestCase.scala         |   8 +-
 .../TestLoadTableConcurrentScenario.scala       |  78 -----
 .../createTable/TestCreateTableAsSelect.scala   |   2 +-
 .../datacompaction/DataCompactionLockTest.scala |   6 +-
 .../MajorCompactionIgnoreInMinorTest.scala      |  12 +-
 .../dataload/TestBatchSortDataLoad.scala        |   5 +-
 .../dataload/TestDataLoadWithFileName.scala     |   5 +-
 .../dataload/TestGlobalSortDataLoad.scala       |   8 +-
 .../dataretention/DataRetentionTestCase.scala   |   6 +-
 .../TestDataLoadingForPartitionTable.scala      |   4 +-
 .../StandardPartitionTableCleanTestCase.scala   |   4 +-
 ...andardPartitionTableCompactionTestCase.scala |   4 +-
 .../StandardPartitionTableLoadingTestCase.scala |   8 +-
 .../org/apache/carbondata/api/CarbonStore.scala |   4 +-
 .../carbondata/spark/CarbonSparkFactory.scala   |   2 +-
 .../spark/DictionaryDetailHelper.scala          |   9 +-
 .../spark/rdd/AlterTableAddColumnRDD.scala      |  17 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |   6 -
 .../carbondata/spark/util/CommonUtil.scala      |  30 +-
 .../carbondata/spark/util/DataLoadingUtil.scala |  10 +-
 .../spark/util/GlobalDictionaryUtil.scala       |  16 +-
 .../command/carbonTableSchemaCommon.scala       |  10 +-
 .../org/apache/spark/util/PartitionUtils.scala  |   6 +-
 .../spark/rdd/AggregateDataMapCompactor.scala   |  13 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  14 +-
 .../spark/rdd/CarbonTableCompactor.scala        |   8 +-
 .../CarbonAlterTableCompactionCommand.scala     |  14 +-
 .../management/CarbonLoadDataCommand.scala      |  14 +-
 .../management/CarbonShowLoadsCommand.scala     |   2 +-
 .../management/RefreshCarbonTableCommand.scala  |  17 +-
 .../command/mutation/DeleteExecution.scala      |   5 +-
 .../CarbonAlterTableDropPartitionCommand.scala  |  18 +-
 .../CarbonAlterTableSplitPartitionCommand.scala |  22 +-
 .../CreatePreAggregateTableCommand.scala        |   2 +-
 .../preaaggregate/PreAggregateUtil.scala        |  15 +-
 .../CarbonAlterTableAddColumnCommand.scala      |   8 +-
 .../CarbonAlterTableDataTypeChangeCommand.scala |   6 +-
 .../CarbonAlterTableDropColumnCommand.scala     |   6 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |  28 +-
 .../spark/sql/hive/CarbonFileMetastore.scala    |  27 +-
 .../spark/sql/hive/CarbonHiveMetaStore.scala    |  13 +-
 .../apache/spark/sql/hive/CarbonMetaStore.scala |   2 +-
 .../apache/spark/sql/hive/CarbonRelation.scala  |   6 +-
 .../org/apache/spark/util/AlterTableUtil.scala  |  54 ++--
 .../partition/TestAlterPartitionTable.scala     |   4 +-
 .../spark/util/AllDictionaryTestCase.scala      |   6 +-
 .../spark/util/DictionaryTestCaseUtil.scala     |   2 +-
 .../util/ExternalColumnDictionaryTestCase.scala |   6 +-
 .../TestStreamingTableOperation.scala           |  24 +-
 .../restructure/AlterTableRevertTestCase.scala  |   2 +-
 .../loading/TableProcessingOperations.java      |  59 ++--
 .../converter/impl/FieldEncoderFactory.java     |   9 +-
 .../merger/AbstractResultProcessor.java         |   8 +-
 .../processing/merger/CarbonCompactionUtil.java |   2 +-
 .../processing/merger/CarbonDataMergerUtil.java |  84 ++----
 .../merger/CompactionResultSortProcessor.java   |   3 +-
 .../merger/RowResultMergerProcessor.java        |   3 +-
 .../store/CarbonFactDataHandlerModel.java       |   9 +-
 .../util/CarbonDataProcessorUtil.java           |  16 +-
 .../processing/util/CarbonLoaderUtil.java       |  95 +++---
 .../processing/util/DeleteLoadFolders.java      |   8 +-
 .../carbon/datastore/BlockIndexStoreTest.java   |  36 +--
 .../carbondata/processing/StoreCreator.java     |  34 +--
 .../streaming/segment/StreamSegment.java        |  24 +-
 .../carbondata/streaming/StreamHandoffRDD.scala |  22 +-
 .../streaming/StreamSinkFactory.scala           |  14 +-
 .../streaming/CarbonAppendableStreamSink.scala  |  18 +-
 103 files changed, 674 insertions(+), 1486 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 f5652ac..a7b47c9 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
@@ -22,7 +22,6 @@ import java.io.Serializable;
 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.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 /**
@@ -42,8 +41,6 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
    */
   private ColumnIdentifier columnIdentifier;
 
-  private transient CarbonTablePath carbonTablePath;
-
   private DataType dataType;
 
   private String dictionaryLocation;
@@ -68,8 +65,7 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
     this.columnIdentifier = columnIdentifier;
     this.dataType = columnIdentifier.getDataType();
     this.dictionaryLocation =
-        CarbonStorePath.getCarbonTablePath(dictionarySourceAbsoluteTableIdentifier)
-            .getMetadataDirectoryPath();
+        CarbonTablePath.getMetadataPath(dictionarySourceAbsoluteTableIdentifier.getTablePath());
   }
 
   /**
@@ -100,13 +96,6 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
     return dataType;
   }
 
-  public CarbonTablePath getCarbonTablePath() {
-    if (null == carbonTablePath) {
-      carbonTablePath = CarbonStorePath.getCarbonTablePath(dictionarySourceAbsoluteTableIdentifier);
-    }
-    return carbonTablePath;
-  }
-
   /**
    * @return columnIdentifier
    */
@@ -118,24 +107,24 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
    * @return dictionary file path
    */
   public String getDictionaryFilePath() {
-    return getCarbonTablePath()
-        .getDictionaryFilePath(dictionaryLocation, columnIdentifier.getColumnId());
+    return CarbonTablePath.getExternalDictionaryFilePath(
+        dictionaryLocation, columnIdentifier.getColumnId());
   }
 
   /**
    * @return dictionary metadata file path
    */
   public String getDictionaryMetaFilePath() {
-    return getCarbonTablePath()
-        .getDictionaryMetaFilePath(dictionaryLocation, columnIdentifier.getColumnId());
+    return CarbonTablePath.getExternalDictionaryMetaFilePath(
+        dictionaryLocation, columnIdentifier.getColumnId());
   }
 
   /**
    * @return sort index file path
    */
   public String getSortIndexFilePath() {
-    return getCarbonTablePath()
-        .getSortIndexFilePath(dictionaryLocation, columnIdentifier.getColumnId());
+    return CarbonTablePath.getExternalSortIndexFilePath(
+        dictionaryLocation, columnIdentifier.getColumnId());
   }
 
   /**
@@ -143,8 +132,8 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
    * @return sort index file path with given offset
    */
   public String getSortIndexFilePath(long offset) {
-    return getCarbonTablePath()
-        .getSortIndexFilePath(dictionaryLocation, columnIdentifier.getColumnId(), offset);
+    return CarbonTablePath.getExternalSortIndexFilePath(
+        dictionaryLocation, columnIdentifier.getColumnId(), offset);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 4f8f724..a7d6027 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
@@ -36,7 +36,6 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 /**
@@ -56,13 +55,11 @@ public class ManageDictionaryAndBTree {
    * clear the dictionary cache
    *
    * @param columnSchema
-   * @param carbonTableIdentifier
+   * @param identifier
    */
   public static void deleteDictionaryFileAndCache(final ColumnSchema columnSchema,
-      AbsoluteTableIdentifier carbonTableIdentifier) {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonTableIdentifier);
-    String metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath();
+      AbsoluteTableIdentifier identifier) {
+    String metadataDirectoryPath = CarbonTablePath.getMetadataPath(identifier.getTablePath());
     CarbonFile metadataDir = FileFactory
         .getCarbonFile(metadataDirectoryPath, FileFactory.getFileType(metadataDirectoryPath));
     if (metadataDir.exists()) {
@@ -90,7 +87,7 @@ public class ManageDictionaryAndBTree {
       }
     }
     // remove dictionary cache
-    removeDictionaryColumnFromCache(carbonTableIdentifier, columnSchema.getColumnUniqueId());
+    removeDictionaryColumnFromCache(identifier, columnSchema.getColumnUniqueId());
   }
 
   /**
@@ -101,7 +98,7 @@ public class ManageDictionaryAndBTree {
   public static void clearBTreeAndDictionaryLRUCache(CarbonTable carbonTable) {
     // clear Btree cache from LRU cache
     LoadMetadataDetails[] loadMetadataDetails =
-        SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath());
+        SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
     String[] segments = new String[loadMetadataDetails.length];
     int i = 0;
     for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 6ef2671..5c2f494 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
@@ -36,7 +36,6 @@ public class AbsoluteTableIdentifier implements Serializable {
    */
   private String tablePath;
 
-  private boolean isLocalPath;
 
   /**
    * carbon table identifier which will have table name and table database
@@ -47,7 +46,6 @@ public class AbsoluteTableIdentifier implements Serializable {
   private AbsoluteTableIdentifier(String tablePath, CarbonTableIdentifier carbonTableIdentifier) {
     //TODO this should be moved to common place where path handling will be handled
     this.tablePath = FileFactory.getUpdatedFilePath(tablePath);
-    isLocalPath = tablePath.startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX);
     this.carbonTableIdentifier = carbonTableIdentifier;
   }
 
@@ -79,7 +77,7 @@ public class AbsoluteTableIdentifier implements Serializable {
   }
 
   public String appendWithLocalPrefix(String path) {
-    if (isLocalPath) {
+    if (tablePath.startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX)) {
       return CarbonCommonConstants.LOCAL_FILE_PREFIX + path;
     } else {
       return path;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java b/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
index 43310fe..2744fcd 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
@@ -48,7 +48,6 @@ import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataFileFooterConverter;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.CarbonIndexFileMergeWriter;
 
@@ -343,13 +342,8 @@ public class PartitionMapFileStore {
       CarbonTable table,
       List<String> currentPartitions,
       boolean forceDelete) throws IOException {
-    SegmentStatusManager ssm = new SegmentStatusManager(table.getAbsoluteTableIdentifier());
-
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(table.getAbsoluteTableIdentifier().getTablePath(),
-            table.getAbsoluteTableIdentifier().getCarbonTableIdentifier());
-
-    LoadMetadataDetails[] details = ssm.readLoadMetadata(table.getMetaDataFilepath());
+    LoadMetadataDetails[] details =
+        SegmentStatusManager.readLoadMetadata(table.getMetadataPath());
     // scan through each segment.
     List<String> segmentsNeedToBeDeleted = new ArrayList<>();
     for (LoadMetadataDetails segment : details) {
@@ -363,7 +357,8 @@ public class PartitionMapFileStore {
         List<String> toBeDeletedIndexFiles = new ArrayList<>();
         List<String> toBeDeletedDataFiles = new ArrayList<>();
         // take the list of files from this segment.
-        String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segment.getLoadName());
+        String segmentPath = CarbonTablePath.getSegmentPath(
+            table.getAbsoluteTableIdentifier().getTablePath(), segment.getLoadName());
         String partitionFilePath = getPartitionFilePath(segmentPath);
         if (partitionFilePath != null) {
           PartitionMapper partitionMapper = readPartitionMap(partitionFilePath);
@@ -436,7 +431,7 @@ public class PartitionMapFileStore {
         SegmentStatusManager.updateDeletionStatus(
             table.getAbsoluteTableIdentifier(),
             segmentsNeedToBeDeleted,
-            table.getMetaDataFilepath());
+            table.getMetadataPath());
       } catch (Exception e) {
         throw new IOException(e);
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 d5c2609..c9d6aad 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
@@ -424,13 +424,20 @@ public class CarbonTable implements Serializable {
   }
 
   /**
-   * @return the metaDataFilepath
+   * Return the metadata path of the table
    */
-  public String getMetaDataFilepath() {
+  public String getMetadataPath() {
     return CarbonTablePath.getMetadataPath(getTablePath());
   }
 
   /**
+   * Return the segment path of the specified segmentId
+   */
+  public String getSemgentPath(String segmentId) {
+    return CarbonTablePath.getSegmentPath(getTablePath(), segmentId);
+  }
+
+  /**
    * @return storepath
    */
   public String getTablePath() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 920155b..a564e42 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
@@ -42,10 +42,8 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
 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;
 
-
 /**
  * This class contains all update utility methods
  */
@@ -182,20 +180,11 @@ public class CarbonUpdateUtil {
                                                   CarbonTable table, String updatedTimeStamp,
                                                   boolean isTimestampUpdationRequired,
                                                   List<String> segmentsToBeDeleted) {
-
     boolean status = false;
-
-    String metaDataFilepath = table.getMetaDataFilepath();
-
-    AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier();
-
-    CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-                    absoluteTableIdentifier.getCarbonTableIdentifier());
-
-    String tableStatusPath = carbonTablePath.getTableStatusFilePath();
-
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    String metaDataFilepath = table.getMetadataPath();
+    AbsoluteTableIdentifier identifier = table.getAbsoluteTableIdentifier();
+    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
 
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     boolean lockStatus = false;
@@ -207,7 +196,7 @@ public class CarbonUpdateUtil {
                         + " for table status updation");
 
         LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-                segmentStatusManager.readLoadMetadata(metaDataFilepath);
+                SegmentStatusManager.readLoadMetadata(metaDataFilepath);
 
         for (LoadMetadataDetails loadMetadata : listOfLoadFolderDetailsArray) {
 
@@ -290,12 +279,8 @@ public class CarbonUpdateUtil {
    */
   public static void cleanStaleDeltaFiles(CarbonTable table, final String timeStamp) {
 
-    AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier();
-
-    CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-                    absoluteTableIdentifier.getCarbonTableIdentifier());
-    String partitionDir = carbonTablePath.getPartitionDir();
+    AbsoluteTableIdentifier identifier = table.getAbsoluteTableIdentifier();
+    String partitionDir = CarbonTablePath.getPartitionDir(identifier.getTablePath());
     CarbonFile file =
             FileFactory.getCarbonFile(partitionDir, FileFactory.getFileType(partitionDir));
     if (!file.exists()) {
@@ -365,8 +350,18 @@ public class CarbonUpdateUtil {
         .substring(0, completeBlockName.lastIndexOf(CarbonCommonConstants.HYPHEN));
   }
 
-  public static long getLatestTaskIdForSegment(String segmentId, CarbonTablePath tablePath) {
-    String segmentDirPath = tablePath.getCarbonDataDirectoryPath(segmentId);
+  /**
+   * returns segment id from segment name
+   *
+   * @param segmentName
+   * @return
+   */
+  public static String getSegmentId(String segmentName) {
+    return segmentName.split(CarbonCommonConstants.UNDERSCORE)[1];
+  }
+
+  public static long getLatestTaskIdForSegment(String segmentId, String tablePath) {
+    String segmentDirPath = CarbonTablePath.getSegmentPath(tablePath, segmentId);
 
     // scan all the carbondata files and get the latest task ID.
     CarbonFile segment =
@@ -405,11 +400,8 @@ public class CarbonUpdateUtil {
 
     SegmentStatusManager ssm = new SegmentStatusManager(table.getAbsoluteTableIdentifier());
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(table.getAbsoluteTableIdentifier().getTablePath(),
-                    table.getAbsoluteTableIdentifier().getCarbonTableIdentifier());
-
-    LoadMetadataDetails[] details = ssm.readLoadMetadata(table.getMetaDataFilepath());
+    LoadMetadataDetails[] details =
+        SegmentStatusManager.readLoadMetadata(table.getMetadataPath());
 
     String validUpdateStatusFile = "";
 
@@ -431,7 +423,8 @@ public class CarbonUpdateUtil {
               || segment.getSegmentStatus() == SegmentStatus.LOAD_PARTIAL_SUCCESS) {
 
         // take the list of files from this segment.
-        String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segment.getLoadName());
+        String segmentPath = CarbonTablePath.getSegmentPath(
+            table.getAbsoluteTableIdentifier().getTablePath(), segment.getLoadName());
         CarbonFile segDir =
                 FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
         CarbonFile[] allSegmentFiles = segDir.listFiles();
@@ -538,8 +531,10 @@ public class CarbonUpdateUtil {
       final String updateStatusTimestamp = validUpdateStatusFile
               .substring(validUpdateStatusFile.lastIndexOf(CarbonCommonConstants.HYPHEN) + 1);
 
-      CarbonFile metaFolder = FileFactory.getCarbonFile(carbonTablePath.getMetadataDirectoryPath(),
-              FileFactory.getFileType(carbonTablePath.getMetadataDirectoryPath()));
+      String tablePath = table.getAbsoluteTableIdentifier().getTablePath();
+      CarbonFile metaFolder = FileFactory.getCarbonFile(
+          CarbonTablePath.getMetadataPath(tablePath),
+          FileFactory.getFileType(CarbonTablePath.getMetadataPath(tablePath)));
 
       CarbonFile[] invalidUpdateStatusFiles = metaFolder.listFiles(new CarbonFileFilter() {
         @Override public boolean accept(CarbonFile file) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 1e73dbb..92c46dd 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
@@ -73,7 +73,7 @@ import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 import org.apache.commons.lang3.ArrayUtils;
 
@@ -292,10 +292,8 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
         .createDimensionInfoAndGetCurrentBlockQueryDimension(blockExecutionInfo,
             queryModel.getProjectionDimensions(), tableBlockDimensions,
             segmentProperties.getComplexDimensions());
-    int tableFactPathLength = CarbonStorePath
-        .getCarbonTablePath(queryModel.getAbsoluteTableIdentifier().getTablePath(),
-            queryModel.getAbsoluteTableIdentifier().getCarbonTableIdentifier()).getFactDir()
-        .length() + 1;
+    int tableFactPathLength = CarbonTablePath.getFactDir(
+        queryModel.getAbsoluteTableIdentifier().getTablePath()).length() + 1;
     blockExecutionInfo.setBlockId(filePath.substring(tableFactPathLength));
     blockExecutionInfo.setDeleteDeltaFilePath(deleteDeltaFiles);
     blockExecutionInfo.setStartBlockletIndex(startBlockletIndex);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 00cdfe7..1765efa 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
@@ -63,7 +63,6 @@ import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnRes
 import org.apache.carbondata.core.scan.model.ProjectionDimension;
 import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 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.commons.lang3.ArrayUtils;
@@ -409,7 +408,7 @@ public class QueryUtil {
   }
 
   public static AbsoluteTableIdentifier getTableIdentifierForColumn(CarbonDimension carbonDimension,
-      AbsoluteTableIdentifier absoluteTableIdentifier) {
+      AbsoluteTableIdentifier identifier) {
     RelationIdentifier relation = carbonDimension.getColumnSchema()
         .getParentColumnTableRelations()
         .get(0)
@@ -417,8 +416,8 @@ public class QueryUtil {
     String parentTableName = relation.getTableName();
     String parentDatabaseName = relation.getDatabaseName();
     String parentTableId = relation.getTableId();
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    String newTablePath = CarbonUtil.getNewTablePath(carbonTablePath, parentTableName);
+    String newTablePath =
+        CarbonTablePath.getNewTablePath(identifier.getTablePath(), parentTableName);
     return AbsoluteTableIdentifier.from(newTablePath, parentDatabaseName, parentTableName,
         parentTableId);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java b/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java
index 6fd9735..3ddf00b 100644
--- a/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java
@@ -16,9 +16,7 @@
  */
 package org.apache.carbondata.core.service;
 
-import org.apache.carbondata.core.service.impl.ColumnUniqueIdGenerator;
 import org.apache.carbondata.core.service.impl.DictionaryFactory;
-import org.apache.carbondata.core.service.impl.PathFactory;
 
 /**
  * Interface to get services
@@ -32,18 +30,4 @@ public class CarbonCommonFactory {
     return DictionaryFactory.getInstance();
   }
 
-  /**
-   * @return path service
-   */
-  public static PathService getPathService() {
-    return PathFactory.getInstance();
-  }
-
-  /**
-   * @return unique id generator
-   */
-  public static ColumnUniqueIdService getColumnUniqueIdGenerator() {
-    return ColumnUniqueIdGenerator.getInstance();
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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
deleted file mode 100644
index 664d045..0000000
--- a/core/src/main/java/org/apache/carbondata/core/service/PathService.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.service;
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-/**
- * Create helper to get path details
- */
-public interface PathService {
-
-  /**
-   * @param absoluteTableIdentifier
-   * @param dictionaryColumnUniqueIdentifier
-   * @return store path related to tables
-   */
-  CarbonTablePath getCarbonTablePath(AbsoluteTableIdentifier absoluteTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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
deleted file mode 100644
index cb4ca25..0000000
--- a/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.service.impl;
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-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;
-
-/**
- * Create helper to get path details
- */
-public class PathFactory implements PathService {
-
-  private static PathService pathService = new PathFactory();
-
-  /**
-   * @param absoluteTableIdentifier
-   * @param
-   * @return store path related to tables
-   */
-  @Override public CarbonTablePath getCarbonTablePath(
-      AbsoluteTableIdentifier absoluteTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    if (null != dictionaryColumnUniqueIdentifier && null != dictionaryColumnUniqueIdentifier
-        .getCarbonTablePath()) {
-      return dictionaryColumnUniqueIdentifier.getCarbonTablePath();
-    }
-    return CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-  }
-
-  public static PathService getInstance() {
-    return pathService;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 e02f246..c613735 100755
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -46,7 +46,6 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 import com.google.gson.Gson;
@@ -59,10 +58,10 @@ public class SegmentStatusManager {
   private static final LogService LOG =
       LogServiceFactory.getLogService(SegmentStatusManager.class.getName());
 
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
+  private AbsoluteTableIdentifier identifier;
 
-  public SegmentStatusManager(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
+  public SegmentStatusManager(AbsoluteTableIdentifier identifier) {
+    this.identifier = identifier;
   }
 
   /**
@@ -71,7 +70,7 @@ public class SegmentStatusManager {
    * @return
    */
   public ICarbonLock getTableStatusLock() {
-    return CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier, LockUsage.TABLE_STATUS_LOCK);
+    return CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.TABLE_STATUS_LOCK);
   }
 
   /**
@@ -79,9 +78,7 @@ public class SegmentStatusManager {
    */
   public static long getTableStatusLastModifiedTime(AbsoluteTableIdentifier identifier)
       throws IOException {
-    String tableStatusPath = CarbonStorePath
-        .getCarbonTablePath(identifier.getTablePath(), identifier.getCarbonTableIdentifier())
-        .getTableStatusFilePath();
+    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
     if (!FileFactory.isFileExist(tableStatusPath, FileFactory.getFileType(tableStatusPath))) {
       return 0L;
     } else {
@@ -103,10 +100,7 @@ public class SegmentStatusManager {
     List<String> listOfValidUpdatedSegments = new ArrayList<>(10);
     List<String> listOfInvalidSegments = new ArrayList<>(10);
     List<String> listOfStreamSegments = new ArrayList<>(10);
-    CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-                    absoluteTableIdentifier.getCarbonTableIdentifier());
-    String dataPath = carbonTablePath.getTableStatusFilePath();
+    String dataPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
     DataInputStream dataInputStream = null;
 
     // Use GSON to deserialize the load information
@@ -280,9 +274,7 @@ public class SegmentStatusManager {
       if (carbonDeleteSegmentLock.lockWithRetries()) {
         LOG.info("Delete segment lock has been successfully acquired");
 
-        CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(
-            identifier.getTablePath(), identifier.getCarbonTableIdentifier());
-        String dataLoadLocation = carbonTablePath.getTableStatusFilePath();
+        String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
         LoadMetadataDetails[] listOfLoadFolderDetailsArray = null;
         if (!FileFactory.isFileExist(dataLoadLocation, FileFactory.getFileType(dataLoadLocation))) {
           // log error.
@@ -362,9 +354,7 @@ public class SegmentStatusManager {
       if (carbonDeleteSegmentLock.lockWithRetries()) {
         LOG.info("Delete segment lock has been successfully acquired");
 
-        CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(
-            identifier.getTablePath(), identifier.getCarbonTableIdentifier());
-        String dataLoadLocation = carbonTablePath.getTableStatusFilePath();
+        String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
         LoadMetadataDetails[] listOfLoadFolderDetailsArray = null;
 
         if (!FileFactory.isFileExist(dataLoadLocation, FileFactory.getFileType(dataLoadLocation))) {
@@ -684,7 +674,7 @@ public class SegmentStatusManager {
    */
   public static Boolean checkIfAnyLoadInProgressForTable(CarbonTable carbonTable) {
     Boolean loadInProgress = false;
-    String metaPath = carbonTable.getMetaDataFilepath();
+    String metaPath = carbonTable.getMetadataPath();
     LoadMetadataDetails[] listOfLoadFolderDetailsArray =
               SegmentStatusManager.readLoadMetadata(metaPath);
     if (listOfLoadFolderDetailsArray.length != 0) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 66f7a12..da70399 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
@@ -48,7 +48,6 @@ import org.apache.carbondata.core.mutate.SegmentUpdateDetails;
 import org.apache.carbondata.core.mutate.TupleIdEnum;
 import org.apache.carbondata.core.mutate.UpdateVO;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 import com.google.gson.Gson;
@@ -64,24 +63,20 @@ public class SegmentUpdateStatusManager {
   private static final LogService LOG =
       LogServiceFactory.getLogService(SegmentUpdateStatusManager.class.getName());
 
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
+  private AbsoluteTableIdentifier identifier;
   private LoadMetadataDetails[] segmentDetails;
   private SegmentUpdateDetails[] updateDetails;
-  private CarbonTablePath carbonTablePath;
   private Map<String, SegmentUpdateDetails> blockAndDetailsMap;
 
   /**
-   * @param absoluteTableIdentifier
+   * @param identifier
    */
-  public SegmentUpdateStatusManager(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
-    carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-        absoluteTableIdentifier.getCarbonTableIdentifier());
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+  public SegmentUpdateStatusManager(AbsoluteTableIdentifier identifier) {
+    this.identifier = identifier;
     // current it is used only for read function scenarios, as file update always requires to work
     // on latest file status.
-    segmentDetails =
-        segmentStatusManager.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());
+    segmentDetails = SegmentStatusManager.readLoadMetadata(
+        CarbonTablePath.getMetadataPath(identifier.getTablePath()));
     updateDetails = readLoadMetadata();
     populateMap();
   }
@@ -128,8 +123,6 @@ public class SegmentUpdateStatusManager {
 
   }
 
-
-
   /**
    * Returns the LoadMetadata Details
    * @return
@@ -160,7 +153,7 @@ public class SegmentUpdateStatusManager {
    * @return
    */
   public ICarbonLock getTableUpdateStatusLock() {
-    return CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier,
+    return CarbonLockFactory.getCarbonLockObj(identifier,
         LockUsage.TABLE_UPDATE_STATUS_LOCK);
   }
 
@@ -176,7 +169,8 @@ public class SegmentUpdateStatusManager {
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     String endTimeStamp = "";
     String startTimeStamp = "";
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
+    String segmentPath = CarbonTablePath.getSegmentPath(
+        identifier.getTablePath(), segmentId);
     CarbonFile segDir =
         FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
     for (LoadMetadataDetails eachSeg : segmentDetails) {
@@ -234,9 +228,7 @@ public class SegmentUpdateStatusManager {
    * @throws Exception
    */
   public String[] getDeleteDeltaFilePath(String blockFilePath) throws Exception {
-    int tableFactPathLength = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier()).getFactDir().length() + 1;
+    int tableFactPathLength = CarbonTablePath.getFactDir(identifier.getTablePath()).length() + 1;
     String blockame = blockFilePath.substring(tableFactPathLength);
     String tupleId = CarbonTablePath.getShortBlockId(blockame);
     return getDeltaFiles(tupleId, CarbonCommonConstants.DELETE_DELTA_FILE_EXT)
@@ -253,11 +245,9 @@ public class SegmentUpdateStatusManager {
    */
   public List<String> getDeltaFiles(String tupleId, String extension) throws Exception {
     try {
-      CarbonTablePath carbonTablePath = CarbonStorePath
-          .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-              absoluteTableIdentifier.getCarbonTableIdentifier());
       String segment = CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID);
-      String carbonDataDirectoryPath = carbonTablePath.getCarbonDataDirectoryPath(segment);
+      String carbonDataDirectoryPath = CarbonTablePath.getSegmentPath(
+          identifier.getTablePath(), segment);
       String completeBlockName = CarbonTablePath.addDataPartPrefix(
           CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.BLOCK_ID)
               + CarbonCommonConstants.FACT_FILE_EXT);
@@ -385,11 +375,8 @@ public class SegmentUpdateStatusManager {
    */
   public CarbonFile[] getDeleteDeltaFilesList(final String segmentId, final String blockName) {
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
+    String segmentPath = CarbonTablePath.getSegmentPath(
+        identifier.getTablePath(), segmentId);
 
     CarbonFile segDir =
         FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
@@ -436,16 +423,12 @@ public class SegmentUpdateStatusManager {
       final String fileExtension, final boolean excludeOriginalFact,
       CarbonFile[] allFilesOfSegment, boolean isAbortedFile) {
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
     String endTimeStamp = "";
     String startTimeStamp = "";
     long factTimeStamp = 0;
 
-    LoadMetadataDetails[] segmentDetails =
-        segmentStatusManager.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());
+    LoadMetadataDetails[] segmentDetails = SegmentStatusManager.readLoadMetadata(
+        CarbonTablePath.getMetadataPath(identifier.getTablePath()));
 
     for (LoadMetadataDetails eachSeg : segmentDetails) {
       if (eachSeg.getLoadName().equalsIgnoreCase(segmentId)) {
@@ -650,13 +633,9 @@ public class SegmentUpdateStatusManager {
       return new SegmentUpdateDetails[0];
     }
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-
     String tableUpdateStatusPath =
-        carbonTablePath.getMetadataDirectoryPath() + CarbonCommonConstants.FILE_SEPARATOR
-            + tableUpdateStatusIdentifier;
+        CarbonTablePath.getMetadataPath(identifier.getTablePath()) +
+            CarbonCommonConstants.FILE_SEPARATOR + tableUpdateStatusIdentifier;
     AtomicFileOperations fileOperation = new AtomicFileOperationsImpl(tableUpdateStatusPath,
         FileFactory.getFileType(tableUpdateStatusPath));
 
@@ -684,12 +663,9 @@ public class SegmentUpdateStatusManager {
    * @return updateStatusFileName
    */
   private String getUpdatedStatusIdentifier() {
-    SegmentStatusManager ssm = new SegmentStatusManager(absoluteTableIdentifier);
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
     LoadMetadataDetails[] loadDetails =
-        ssm.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());
+        SegmentStatusManager.readLoadMetadata(
+            CarbonTablePath.getMetadataPath(identifier.getTablePath()));
     if (loadDetails.length == 0) {
       return null;
     }
@@ -704,13 +680,9 @@ public class SegmentUpdateStatusManager {
    */
   public void writeLoadDetailsIntoFile(List<SegmentUpdateDetails> listOfSegmentUpdateDetailsArray,
       String updateStatusFileIdentifier) throws IOException {
-
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-
     String fileLocation =
-        carbonTablePath.getMetadataDirectoryPath() + CarbonCommonConstants.FILE_SEPARATOR
+        CarbonTablePath.getMetadataPath(identifier.getTablePath())
+            + CarbonCommonConstants.FILE_SEPARATOR
             + CarbonUpdateUtil.getUpdateStatusFileName(updateStatusFileIdentifier);
 
     AtomicFileOperations fileWrite =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 b16bc5e..80a382c 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
@@ -85,7 +85,6 @@ import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.BlockletHeader;
 import org.apache.carbondata.format.DataChunk2;
@@ -1065,20 +1064,18 @@ public final class CarbonUtil {
    *
    * @param taskId
    * @param tableBlockInfoList
-   * @param absoluteTableIdentifier
+   * @param identifier
    */
   public static long calculateDriverBTreeSize(String taskId, String bucketNumber,
-      List<TableBlockInfo> tableBlockInfoList, AbsoluteTableIdentifier absoluteTableIdentifier) {
+      List<TableBlockInfo> tableBlockInfoList, AbsoluteTableIdentifier identifier) {
     // need to sort the  block info list based for task in ascending  order so
     // it will be sinkup with block index read from file
     Collections.sort(tableBlockInfoList);
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
     // geting the index file path
     //TODO need to pass proper partition number when partiton will be supported
-    String carbonIndexFilePath = carbonTablePath
-        .getCarbonIndexFilePath(taskId, tableBlockInfoList.get(0).getSegmentId(),
+    String carbonIndexFilePath = CarbonTablePath
+        .getCarbonIndexFilePath(identifier.getTablePath(), taskId,
+            tableBlockInfoList.get(0).getSegmentId(),
             bucketNumber, CarbonTablePath.DataFileUtil
                 .getTimeStampFromFileName(tableBlockInfoList.get(0).getFilePath()),
             tableBlockInfoList.get(0).getVersion());
@@ -1305,23 +1302,21 @@ public final class CarbonUtil {
    *
    * @param taskId                  task id of the file
    * @param tableBlockInfoList      list of table block
-   * @param absoluteTableIdentifier absolute table identifier
+   * @param identifier absolute table identifier
    * @return list of block info
    * @throws IOException if any problem while reading
    */
   public static List<DataFileFooter> readCarbonIndexFile(String taskId, String bucketNumber,
-      List<TableBlockInfo> tableBlockInfoList, AbsoluteTableIdentifier absoluteTableIdentifier)
+      List<TableBlockInfo> tableBlockInfoList, AbsoluteTableIdentifier identifier)
       throws IOException {
     // need to sort the  block info list based for task in ascending  order so
     // it will be sinkup with block index read from file
     Collections.sort(tableBlockInfoList);
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
     // geting the index file path
     //TODO need to pass proper partition number when partiton will be supported
-    String carbonIndexFilePath = carbonTablePath
-        .getCarbonIndexFilePath(taskId, tableBlockInfoList.get(0).getSegmentId(),
+    String carbonIndexFilePath = CarbonTablePath
+        .getCarbonIndexFilePath(identifier.getTablePath(), taskId,
+            tableBlockInfoList.get(0).getSegmentId(),
             bucketNumber, CarbonTablePath.DataFileUtil
                 .getTimeStampFromFileName(tableBlockInfoList.get(0).getFilePath()),
             tableBlockInfoList.get(0).getVersion());
@@ -2177,21 +2172,6 @@ public final class CarbonUtil {
   }
 
   /**
-   * 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 newTableName          new table name
-   * @return the new table path
-   */
-  public static String getNewTablePath(
-      Path carbonTablePath,
-      String newTableName) {
-    Path parentPath = carbonTablePath.getParent();
-    return parentPath.toString() + CarbonCommonConstants.FILE_SEPARATOR + newTableName;
-  }
-
-  /**
    * This method will calculate the data size and index size for carbon table
    */
   public static Map<String, Long> calculateDataIndexSize(CarbonTable carbonTable)
@@ -2201,18 +2181,17 @@ public final class CarbonUtil {
     long indexSize = 0L;
     long lastUpdateTime = 0L;
     boolean needUpdate = false;
-    AbsoluteTableIdentifier absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier();
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
+    AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
     String isCalculated = CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.ENABLE_CALCULATE_SIZE,
             CarbonCommonConstants.DEFAULT_ENABLE_CALCULATE_SIZE);
     if (isCalculated.equalsIgnoreCase("true")) {
-      SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+      SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
       ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
       try {
         if (carbonLock.lockWithRetries()) {
           LOGGER.info("Acquired lock for table for table status updation");
-          String metadataPath = carbonTable.getMetaDataFilepath();
+          String metadataPath = carbonTable.getMetadataPath();
           LoadMetadataDetails[] loadMetadataDetails =
               SegmentStatusManager.readLoadMetadata(metadataPath);
 
@@ -2226,8 +2205,8 @@ public final class CarbonUtil {
               if (null == dsize || null == isize) {
                 needUpdate = true;
                 LOGGER.info("It is an old segment, need calculate data size and index size again");
-                HashMap<String, Long> map = CarbonUtil
-                    .getDataSizeAndIndexSize(carbonTablePath, loadMetadataDetail.getLoadName());
+                HashMap<String, Long> map = CarbonUtil.getDataSizeAndIndexSize(
+                    identifier.getTablePath(), loadMetadataDetail.getLoadName());
                 dsize = String.valueOf(map.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE));
                 isize = String.valueOf(map.get(CarbonCommonConstants.CARBON_TOTAL_INDEX_SIZE));
                 loadMetadataDetail.setDataSize(dsize);
@@ -2239,10 +2218,12 @@ public final class CarbonUtil {
           }
           // If it contains old segment, write new load details
           if (needUpdate) {
-            SegmentStatusManager.writeLoadDetailsIntoFile(carbonTablePath.getTableStatusFilePath(),
+            SegmentStatusManager.writeLoadDetailsIntoFile(
+                CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()),
                 loadMetadataDetails);
           }
-          String tableStatusPath = carbonTablePath.getTableStatusFilePath();
+          String tableStatusPath =
+              CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
           if (FileFactory.isFileExist(tableStatusPath, FileFactory.getFileType(tableStatusPath))) {
             lastUpdateTime =
                 FileFactory.getCarbonFile(tableStatusPath, FileFactory.getFileType(tableStatusPath))
@@ -2269,12 +2250,12 @@ public final class CarbonUtil {
   }
 
   // Get the total size of carbon data and the total size of carbon index
-  public static HashMap<String, Long> getDataSizeAndIndexSize(CarbonTablePath carbonTablePath,
+  public static HashMap<String, Long> getDataSizeAndIndexSize(String tablePath,
       String segmentId) throws IOException {
     long carbonDataSize = 0L;
     long carbonIndexSize = 0L;
     HashMap<String, Long> dataAndIndexSize = new HashMap<String, Long>();
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
+    String segmentPath = CarbonTablePath.getSegmentPath(tablePath, segmentId);
     FileFactory.FileType fileType = FileFactory.getFileType(segmentPath);
     switch (fileType) {
       case HDFS:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java
deleted file mode 100644
index b864e6e..0000000
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.util.path;
-
-import java.io.File;
-
-/**
- * Helps to get Shared dimension files path.
- */
-public class CarbonSharedDictionaryPath {
-
-  private static final String SHAREDDIM_DIR = "SharedDictionary";
-  private static final String DICTIONARY_EXT = ".dict";
-  private static final String DICTIONARY_META_EXT = ".dictmeta";
-  private static final String SORT_INDEX_EXT = ".sortindex";
-
-  /***
-   * @param storePath    store path
-   * @param databaseName data base name
-   * @param columnId     unique column identifier
-   * @return absolute path of shared dictionary file
-   */
-  public static String getDictionaryFilePath(String storePath, String databaseName,
-      String columnId) {
-    return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId
-        + DICTIONARY_EXT;
-  }
-
-  /***
-   * @param storePath    store path
-   * @param databaseName data base name
-   * @param columnId     unique column identifier
-   * @return absolute path of shared dictionary meta file
-   */
-  public static String getDictionaryMetaFilePath(String storePath, String databaseName,
-      String columnId) {
-    return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId
-        + DICTIONARY_META_EXT;
-  }
-
-  /***
-   * @param storePath    store path
-   * @param databaseName data base name
-   * @param columnId     unique column identifier
-   * @return absolute path of shared dictionary sort index file
-   */
-  public static String getSortIndexFilePath(String storePath, String databaseName,
-      String columnId) {
-    return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId
-        + SORT_INDEX_EXT;
-  }
-
-  private static String getSharedDictionaryDir(String storePath, String databaseName) {
-    return storePath + File.separator + databaseName + File.separator + SHAREDDIM_DIR;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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
deleted file mode 100644
index e57448d..0000000
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.util.path;
-
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-
-import org.apache.hadoop.fs.Path;
-
-/**
- * Helps to get Store content paths.
- */
-public class CarbonStorePath extends Path {
-
-  private String storePath;
-
-  public CarbonStorePath(String storePathString) {
-    super(storePathString);
-    this.storePath = storePathString;
-  }
-
-  /**
-   * gets CarbonTablePath object to manage table paths
-   *
-   * @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 tablePath,
-      CarbonTableIdentifier tableIdentifier) {
-    return new CarbonTablePath(tableIdentifier, tablePath);
-  }
-
-  public static CarbonTablePath getCarbonTablePath(AbsoluteTableIdentifier identifier) {
-    CarbonTableIdentifier id = identifier.getCarbonTableIdentifier();
-    return new CarbonTablePath(id, identifier.getTablePath());
-  }
-
-  /**
-   * gets CarbonTablePath object to manage table paths
-   */
-  public CarbonTablePath getCarbonTablePath(CarbonTableIdentifier tableIdentifier) {
-    return CarbonStorePath.getCarbonTablePath(storePath, tableIdentifier);
-  }
-
-  @Override public boolean equals(Object o) {
-    if (!(o instanceof CarbonStorePath)) {
-      return false;
-    }
-    CarbonStorePath path = (CarbonStorePath)o;
-    return storePath.equals(path.storePath) && super.equals(o);
-  }
-
-  @Override public int hashCode() {
-    return super.hashCode() + storePath.hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 36eaa56..d870f66 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
@@ -22,16 +22,14 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 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.ColumnarFormatVersion;
 
 import org.apache.hadoop.fs.Path;
 
-
 /**
  * Helps to get Table content paths.
  */
-public class CarbonTablePath extends Path {
+public class CarbonTablePath {
 
   private static final String METADATA_DIR = "Metadata";
   private static final String DICTIONARY_EXT = ".dict";
@@ -54,19 +52,10 @@ public class CarbonTablePath extends Path {
   private static final String STREAMING_LOG_DIR = "log";
   private static final String STREAMING_CHECKPOINT_DIR = "checkpoint";
 
-  private String tablePath;
-  private CarbonTableIdentifier carbonTableIdentifier;
-
   /**
-   * structure CarbonTablePath object to manage table paths
-   *
-   * @param carbonTableIdentifier identifier of carbon table that the segment belong to
-   * @param tablePathString the store path of the segment
+   * This class provides static utility only.
    */
-  public CarbonTablePath(CarbonTableIdentifier carbonTableIdentifier, String tablePathString) {
-    super(tablePathString);
-    this.carbonTableIdentifier = carbonTableIdentifier;
-    this.tablePath = tablePathString;
+  private CarbonTablePath() {
   }
 
   /**
@@ -136,120 +125,74 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * gets table path
-   */
-  public String getPath() {
-    return tablePath;
-  }
-
-  /**
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
+   * Return absolute path of dictionary file
    */
-  public String getDictionaryFilePath(String columnId) {
-    return getMetaDataDir() + File.separator + getDictionaryFileName(columnId);
+  public static String getDictionaryFilePath(String tablePath, String columnId) {
+    return getMetadataPath(tablePath) + File.separator + getDictionaryFileName(columnId);
   }
 
   /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
+   * Return absolute path of dictionary file
    */
-  public String getDictionaryFilePath(String dictionaryPath, String columnId) {
+  public static String getExternalDictionaryFilePath(String dictionaryPath, String columnId) {
     return dictionaryPath + File.separator + getDictionaryFileName(columnId);
   }
 
   /**
-   * This method will return the metadata directory location for a table
-   *
-   * @return
-   */
-  public String getMetadataDirectoryPath() {
-    return getMetaDataDir();
-  }
-
-  /**
-   * Return metadata path based on `tablePath`
+   * Return metadata path
    */
   public static String getMetadataPath(String tablePath) {
     return tablePath + File.separator + METADATA_DIR;
   }
 
   /**
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary meta file
-   */
-  public String getDictionaryMetaFilePath(String columnId) {
-    return getMetaDataDir() + File.separator + columnId + DICTIONARY_META_EXT;
-  }
-
-  /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
+   * Return absolute path of dictionary meta file
    */
-  public String getDictionaryMetaFilePath(String dictionaryPath, String columnId) {
+  public static String getExternalDictionaryMetaFilePath(String dictionaryPath, String columnId) {
     return dictionaryPath + File.separator + columnId + DICTIONARY_META_EXT;
   }
 
   /**
-   * @param columnId unique column identifier
-   * @return absolute path of sort index file
+   * Return absolute path of dictionary meta file
    */
-  public String getSortIndexFilePath(String columnId) {
-    return getMetaDataDir() + File.separator + columnId + SORT_INDEX_EXT;
+  public static String getDictionaryMetaFilePath(String tablePath, String columnId) {
+    return getMetadataPath(tablePath) + File.separator + columnId + DICTIONARY_META_EXT;
   }
 
   /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
+   * Return absolute path of sort index file
    */
-  public String getSortIndexFilePath(String dictionaryPath, String columnId) {
-    return dictionaryPath + File.separator + columnId + SORT_INDEX_EXT;
+  public static String getSortIndexFilePath(String tablePath, String columnId) {
+    return getMetadataPath(tablePath) + File.separator + columnId + SORT_INDEX_EXT;
   }
 
   /**
-   *
-   * @param columnId
-   * @param dictOffset
-   * @return absolute path of sortindex with appeneded dictionary offset
+   * Return sortindex file path based on specified dictionary path
    */
-  public String getSortIndexFilePath(String columnId, long dictOffset) {
-    return getMetaDataDir() + File.separator + columnId + "_" + dictOffset + SORT_INDEX_EXT;
+  public static String getExternalSortIndexFilePath(String dictionaryPath, String columnId) {
+    return dictionaryPath + File.separator + columnId + SORT_INDEX_EXT;
   }
 
   /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @param dictOffset
-   * @return absolute path of dictionary file
+   * Return sortindex file path for columnId and offset based on specified dictionary path
    */
-  public String getSortIndexFilePath(String dictionaryPath, String columnId, long dictOffset) {
+  public static String getExternalSortIndexFilePath(String dictionaryPath, String columnId,
+      long dictOffset) {
     return dictionaryPath + File.separator + columnId + "_" + dictOffset + SORT_INDEX_EXT;
   }
 
   /**
-   * @return absolute path of schema file
-   */
-  public String getSchemaFilePath() {
-    return getMetaDataDir() + File.separator + SCHEMA_FILE;
-  }
-
-  /**
-   * return the schema file path
-   * @param tablePath path to table files
-   * @return schema file path
+   * Return absolute path of schema file
    */
   public static String getSchemaFilePath(String tablePath) {
-    return tablePath + File.separator + METADATA_DIR + File.separator + SCHEMA_FILE;
+    return getMetadataPath(tablePath) + File.separator + SCHEMA_FILE;
   }
 
   /**
-   * @return absolute path of table status file
+   * Return absolute path of table status file
    */
-  public String getTableStatusFilePath() {
-    return getMetaDataDir() + File.separator + TABLE_STATUS_FILE;
+  public static String getTableStatusFilePath(String tablePath) {
+    return getMetadataPath(tablePath) + File.separator + TABLE_STATUS_FILE;
   }
 
   /**
@@ -260,9 +203,9 @@ public class CarbonTablePath extends Path {
    * @param factUpdateTimeStamp unique identifier to identify an update
    * @return absolute path of data file stored in carbon data format
    */
-  public String getCarbonDataFilePath(String segmentId, Integer filePartNo, Long taskNo,
-      int batchNo, int bucketNumber, String factUpdateTimeStamp) {
-    return getSegmentDir(segmentId) + File.separator + getCarbonDataFileName(
+  public static String getCarbonDataFilePath(String tablePath, String segmentId, Integer filePartNo,
+      Long taskNo, int batchNo, int bucketNumber, String factUpdateTimeStamp) {
+    return getSegmentPath(tablePath, segmentId) + File.separator + getCarbonDataFileName(
         filePartNo, taskNo, bucketNumber, batchNo, factUpdateTimeStamp);
   }
 
@@ -274,9 +217,9 @@ public class CarbonTablePath extends Path {
    * @param segmentId   segment number
    * @return full qualified carbon index path
    */
-  public String getCarbonIndexFilePath(final String taskId, final String segmentId,
-      final String bucketNumber) {
-    String segmentDir = getSegmentDir(segmentId);
+  private static String getCarbonIndexFilePath(final String tablePath, final String taskId,
+      final String segmentId, final String bucketNumber) {
+    String segmentDir = getSegmentPath(tablePath, segmentId);
     CarbonFile carbonFile =
         FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir));
 
@@ -309,27 +252,28 @@ public class CarbonTablePath extends Path {
    *        timestamp
    * @return carbon index file path
    */
-  public String getCarbonIndexFilePath(String taskId, String segmentId, String bucketNumber,
-      String timeStamp, ColumnarFormatVersion columnarFormatVersion) {
+  public static String getCarbonIndexFilePath(String tablePath, String taskId, String segmentId,
+      String bucketNumber, String timeStamp, ColumnarFormatVersion columnarFormatVersion) {
     switch (columnarFormatVersion) {
       case V1:
       case V2:
-        return getCarbonIndexFilePath(taskId, segmentId, bucketNumber);
+        return getCarbonIndexFilePath(tablePath, taskId, segmentId, bucketNumber);
       default:
-        String segmentDir = getSegmentDir(segmentId);
+        String segmentDir = getSegmentPath(tablePath, segmentId);
         return segmentDir + File.separator + getCarbonIndexFileName(taskId,
             Integer.parseInt(bucketNumber), timeStamp);
     }
   }
 
-  public String getCarbonIndexFilePath(String taskId, String segmentId, int batchNo,
-      String bucketNumber, String timeStamp, ColumnarFormatVersion columnarFormatVersion) {
+  public static String getCarbonIndexFilePath(String tablePath, String taskId, String segmentId,
+      int batchNo, String bucketNumber, String timeStamp,
+      ColumnarFormatVersion columnarFormatVersion) {
     switch (columnarFormatVersion) {
       case V1:
       case V2:
-        return getCarbonIndexFilePath(taskId, segmentId, bucketNumber);
+        return getCarbonIndexFilePath(tablePath, taskId, segmentId, bucketNumber);
       default:
-        String segmentDir = getSegmentDir(segmentId);
+        String segmentDir = getSegmentPath(tablePath, segmentId);
         return segmentDir + File.separator + getCarbonIndexFileName(Long.parseLong(taskId),
             Integer.parseInt(bucketNumber), batchNo, timeStamp);
     }
@@ -341,13 +285,10 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * Gets absolute path of data file
-   *
-   * @param segmentId   unique partition identifier
-   * @return absolute path of data file stored in carbon data format
+   * Return the segment path from table path and segmentid
    */
-  public String getCarbonDataDirectoryPath(String segmentId) {
-    return getSegmentDir(segmentId);
+  public static String getSegmentPath(String tablePath, String segmentId) {
+    return getPartitionDir(tablePath) + File.separator + SEGMENT_PREFIX + segmentId;
   }
 
   /**
@@ -385,48 +326,46 @@ public class CarbonTablePath extends Path {
     return segmentDir + File.separator + getCarbonStreamIndexFileName();
   }
 
-  public String getSegmentDir(String segmentId) {
-    return getPartitionDir() + File.separator + SEGMENT_PREFIX + segmentId;
-  }
-
   // This partition is not used in any code logic, just keep backward compatibility
   public static final String DEPRECATED_PATITION_ID = "0";
 
-  public String getPartitionDir() {
-    return getFactDir() + File.separator + PARTITION_PREFIX +
-        CarbonTablePath.DEPRECATED_PATITION_ID;
+  /**
+   * Return true if tablePath exists
+   */
+  public static boolean exists(String tablePath) {
+    return FileFactory.getCarbonFile(tablePath, FileFactory.getFileType(tablePath)).exists();
   }
 
-  private String getMetaDataDir() {
-    return tablePath + File.separator + METADATA_DIR;
+  public static String getPartitionDir(String tablePath) {
+    return getFactDir(tablePath) + File.separator + PARTITION_PREFIX +
+        CarbonTablePath.DEPRECATED_PATITION_ID;
   }
 
-  public String getFactDir() {
+  public static String getFactDir(String tablePath) {
     return tablePath + File.separator + FACT_DIR;
   }
 
-  public String getStreamingLogDir() {
+  public static String getStreamingLogDir(String tablePath) {
     return tablePath + File.separator + STREAMING_DIR + File.separator + STREAMING_LOG_DIR;
   }
 
-  public String getStreamingCheckpointDir() {
+  public static String getStreamingCheckpointDir(String tablePath) {
     return tablePath + File.separator + STREAMING_DIR + File.separator + STREAMING_CHECKPOINT_DIR;
   }
 
-  public CarbonTableIdentifier getCarbonTableIdentifier() {
-    return carbonTableIdentifier;
-  }
-
-  @Override public boolean equals(Object o) {
-    if (!(o instanceof CarbonTablePath)) {
-      return false;
-    }
-    CarbonTablePath path = (CarbonTablePath) o;
-    return tablePath.equals(path.tablePath) && super.equals(o);
-  }
-
-  @Override public int hashCode() {
-    return super.hashCode() + tablePath.hashCode();
+  /**
+   * get the parent folder of old table path and returns the new tablePath by appending new
+   * tableName to the parent
+   *
+   * @param tablePath         Old tablePath
+   * @param newTableName      new table name
+   * @return the new table path
+   */
+  public static String getNewTablePath(
+      String tablePath,
+      String newTableName) {
+    Path parentPath = new Path(tablePath).getParent();
+    return parentPath.toString() + CarbonCommonConstants.FILE_SEPARATOR + newTableName;
   }
 
   /**
@@ -445,11 +384,8 @@ public class CarbonTablePath extends Path {
       return fileName.substring(startIndex, endIndex);
     }
 
-
     /**
-     * This will return the timestamp present in the delete delta file.
-     * @param fileName
-     * @return
+     * Return the timestamp present in the delete delta file.
      */
     public static String getTimeStampFromDeleteDeltaFile(String fileName) {
       return fileName.substring(fileName.lastIndexOf(CarbonCommonConstants.HYPHEN) + 1,
@@ -457,9 +393,7 @@ public class CarbonTablePath extends Path {
     }
 
     /**
-     * This will return the timestamp present in the delete delta file.
-     * @param fileName
-     * @return
+     * Return the timestamp present in the delete delta file.
      */
     public static String getBlockNameFromDeleteDeltaFile(String fileName) {
       return fileName.substring(0,
@@ -467,7 +401,7 @@ public class CarbonTablePath extends Path {
     }
 
     /**
-     * gets updated timestamp information from given carbon data file name
+     * Return the updated timestamp information from given carbon data file name
      */
     public static String getBucketNo(String carbonFilePath) {
       // Get the file name from path
@@ -485,7 +419,7 @@ public class CarbonTablePath extends Path {
     }
 
     /**
-     * gets file part number information from given carbon data file name
+     * Return the file part number information from given carbon data file name
      */
     public static String getPartNo(String carbonDataFileName) {
       // Get the file name from path
@@ -497,7 +431,7 @@ public class CarbonTablePath extends Path {
     }
 
     /**
-     * gets updated timestamp information from given carbon data file name
+     * Return the updated timestamp information from given carbon data file name
      */
     public static String getTaskNo(String carbonDataFileName) {
       // Get the file name from path
@@ -510,35 +444,30 @@ public class CarbonTablePath extends Path {
     }
 
     /**
-     * get the taskId part from taskNo(include taskId + batchNo)
-     * @param taskNo
-     * @return
+     * Return the taskId part from taskNo(include taskId + batchNo)
      */
     public static long getTaskIdFromTaskNo(String taskNo) {
       return Long.parseLong(taskNo.split(BATCH_PREFIX)[0]);
     }
 
+    /**
+     * Return the batch number from taskNo string
+     */
     public static int getBatchNoFromTaskNo(String taskNo) {
       return Integer.parseInt(taskNo.split(BATCH_PREFIX)[1]);
     }
 
     /**
-     * Gets the file name from file path
+     * Return the file name from file path
      */
-    private static String getFileName(String carbonDataFileName) {
-      int endIndex = carbonDataFileName.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR);
+    private static String getFileName(String dataFilePath) {
+      int endIndex = dataFilePath.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR);
       if (endIndex > -1) {
-        return carbonDataFileName.substring(endIndex + 1, carbonDataFileName.length());
+        return dataFilePath.substring(endIndex + 1, dataFilePath.length());
       } else {
-        return carbonDataFileName;
+        return dataFilePath;
       }
     }
-  }
-
-  /**
-   * To manage data path and composition
-   */
-  public static class DataPathUtil {
 
     /**
      * gets segement id from given absolute data file path
@@ -546,11 +475,11 @@ public class CarbonTablePath extends Path {
     public static String getSegmentId(String dataFileAbsolutePath) {
       // find segment id from last of data file path
       String tempdataFileAbsolutePath = dataFileAbsolutePath.replace(
-              CarbonCommonConstants.WINDOWS_FILE_SEPARATOR, CarbonCommonConstants.FILE_SEPARATOR);
+          CarbonCommonConstants.WINDOWS_FILE_SEPARATOR, CarbonCommonConstants.FILE_SEPARATOR);
       int endIndex = tempdataFileAbsolutePath.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR);
       // + 1 for size of "/"
       int startIndex = tempdataFileAbsolutePath.lastIndexOf(
-              CarbonCommonConstants.FILE_SEPARATOR, endIndex - 1) + 1;
+          CarbonCommonConstants.FILE_SEPARATOR, endIndex - 1) + 1;
       String segmentDirStr = dataFileAbsolutePath.substring(startIndex, endIndex);
       //identify id in segment_<id>
       String[] segmentDirSplits = segmentDirStr.split("_");
@@ -582,19 +511,16 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * returns the carbondata file name
-   *
-   * @param carbonDataFilePath carbondata file path
-   * @return
+   * Return the carbondata file name
    */
   public static String getCarbonDataFileName(String carbonDataFilePath) {
-    return carbonDataFilePath
-        .substring(carbonDataFilePath.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR) + 1,
-            carbonDataFilePath.indexOf(CARBON_DATA_EXT));
+    return carbonDataFilePath.substring(
+        carbonDataFilePath.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR) + 1,
+        carbonDataFilePath.indexOf(CARBON_DATA_EXT));
   }
 
   /**
-   * @return prefix of carbon data
+   * Return prefix of carbon data
    */
   public static String getCarbonDataPrefix() {
     return DATA_PART_PREFIX;
@@ -638,40 +564,6 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * This method will append strings in path and return block id
-   *
-   * @param shortBlockId
-   * @return blockId
-   */
-  public static String getBlockId(String shortBlockId) {
-    String[] splitRecords = shortBlockId.split(CarbonCommonConstants.FILE_SEPARATOR);
-    StringBuffer sb = new StringBuffer();
-    for (int i = 0; i < splitRecords.length; i++) {
-      if (i == 0) {
-        sb.append(PARTITION_PREFIX);
-        sb.append(splitRecords[i]);
-      } else if (i == 1) {
-        sb.append(CarbonCommonConstants.FILE_SEPARATOR);
-        sb.append(SEGMENT_PREFIX);
-        sb.append(splitRecords[i]);
-      } else if (i == 2) {
-        sb.append(CarbonCommonConstants.FILE_SEPARATOR);
-        sb.append(DATA_PART_PREFIX);
-        sb.append(splitRecords[i]);
-      } else if (i == 3) {
-        sb.append(CarbonCommonConstants.FILE_SEPARATOR);
-        sb.append(splitRecords[i]);
-        sb.append(CARBON_DATA_EXT);
-      } else {
-        sb.append(CarbonCommonConstants.FILE_SEPARATOR);
-        sb.append(splitRecords[i]);
-      }
-    }
-    return sb.toString();
-  }
-
-
-  /**
    * adds data part prefix to given value
    * @return partition prefix
    */
@@ -700,10 +592,4 @@ public class CarbonTablePath extends Path {
         + "-" + DataFileUtil.getTimeStampFromFileName(actualBlockName) + INDEX_FILE_EXT;
   }
 
-  /**
-   * Get the segment path from table path and segmentid
-   */
-  public static String getSegmentPath(String tablePath, String segmentId) {
-    return tablePath + "/Fact/Part0/Segment_" + segmentId;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 31e44a2..7d829b9 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
@@ -34,7 +34,6 @@ 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;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -48,7 +47,7 @@ public class AbstractDictionaryCacheTest {
 
   protected CarbonTableIdentifier carbonTableIdentifier;
 
-  protected AbsoluteTableIdentifier absoluteTableIdentifier;
+  protected AbsoluteTableIdentifier identifier;
 
   protected String databaseName;
 
@@ -107,7 +106,7 @@ public class AbstractDictionaryCacheTest {
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
       String columnId) {
 	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataTypes.STRING);
-    return new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
+    return new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier,
         DataTypes.STRING);
   }
 
@@ -130,13 +129,11 @@ public class AbstractDictionaryCacheTest {
       throws IOException {
 	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
+        new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier,
             columnIdentifier.getDataType());
     CarbonDictionaryWriter carbonDictionaryWriter =
         new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    CarbonUtil.checkAndCreateFolder(carbonTablePath.getMetadataDirectoryPath());
+    CarbonUtil.checkAndCreateFolder(CarbonTablePath.getMetadataPath(identifier.getTablePath()));
     List<byte[]> valueList = convertStringListToByteArray(data);
     try {
       carbonDictionaryWriter.write(valueList);


[3/4] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 d0aedd4..c36c89d 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
@@ -56,7 +56,7 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
     this.carbonStorePath = props.getProperty("storePath", "carbonStore");
     carbonTableIdentifier =
         new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    absoluteTableIdentifier =
+    identifier =
         AbsoluteTableIdentifier.from(carbonStorePath + "/" + databaseName + "/" + tableName,
             carbonTableIdentifier);
     columnIdentifiers = new String[] { "name", "place" };
@@ -67,7 +67,7 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
 
   @After public void tearDown() throws Exception {
     carbonTableIdentifier = null;
-    absoluteTableIdentifier = null;
+    identifier = null;
     forwardDictionaryCache = null;
     deleteStorePath();
   }
@@ -217,7 +217,7 @@ 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(absoluteTableIdentifier, columnIdentifier,
+        new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier,
             columnIdentifier.getDataType());
     Map<String, Integer> dataToSurrogateKeyMap = new HashMap<>(data.size());
     int surrogateKey = 0;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 01cb3a9..d2bf2e3 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
@@ -58,7 +58,7 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
     this.carbonStorePath = props.getProperty("storePath", "carbonStore");
     carbonTableIdentifier =
         new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    absoluteTableIdentifier = AbsoluteTableIdentifier.from(
+    identifier = AbsoluteTableIdentifier.from(
         carbonStorePath + "/" + databaseName + "/" + tableName, carbonTableIdentifier);
     columnIdentifiers = new String[] { "name", "place" };
     deleteStorePath();
@@ -69,7 +69,7 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
   @After public void tearDown() throws Exception {
     carbonTableIdentifier = null;
     reverseDictionaryCache = null;
-    absoluteTableIdentifier = null;
+    identifier = null;
     deleteStorePath();
   }
 
@@ -276,6 +276,6 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
 	      String columnId) {
 	    ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataTypes.DOUBLE);
-    return new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier);
+    return new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier);
 	  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 d3c3bc3..ecabfd4 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
@@ -28,8 +28,6 @@ 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.service.CarbonCommonFactory;
-import org.apache.carbondata.core.service.PathService;
-import org.apache.carbondata.core.service.impl.PathFactory;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -58,12 +56,6 @@ public class CarbonDictionaryReaderImplTest {
   }
 
   @Test public void testRead() throws Exception {
-    new MockUp<CarbonCommonFactory>() {
-      @Mock public PathService getPathService() {
-
-        return new PathFactory();
-      }
-    };
     new MockUp<CarbonDictionaryMetadataReaderImpl>() {
       @Mock public List<CarbonDictionaryColumnMetaChunk> read() throws IOException {
         CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunks =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 a1ccab3..4293536 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
@@ -39,21 +39,19 @@ public class CarbonFormatDirectoryStructureTest {
    */
   @Test public void testTablePathStructure() throws IOException {
     CarbonTableIdentifier tableIdentifier = new CarbonTableIdentifier("d1", "t1", UUID.randomUUID().toString());
-    CarbonStorePath carbonStorePath = new CarbonStorePath(CARBON_STORE);
-    AbsoluteTableIdentifier absoluteTableIdentifier =
+    AbsoluteTableIdentifier identifier =
         AbsoluteTableIdentifier.from(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("\\", "/")
+    assertTrue(identifier.getTablePath().replace("\\", "/").equals(CARBON_STORE + "/d1/t1"));
+    assertTrue(CarbonTablePath.getSchemaFilePath(identifier.getTablePath()).replace("\\", "/").equals(CARBON_STORE + "/d1/t1/Metadata/schema"));
+    assertTrue(CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()).replace("\\", "/")
         .equals(CARBON_STORE + "/d1/t1/Metadata/tablestatus"));
-    assertTrue(carbonTablePath.getDictionaryFilePath("t1_c1").replace("\\", "/")
+    assertTrue(CarbonTablePath.getDictionaryFilePath(identifier.getTablePath(), "t1_c1").replace("\\", "/")
         .equals(CARBON_STORE + "/d1/t1/Metadata/t1_c1.dict"));
-    assertTrue(carbonTablePath.getDictionaryMetaFilePath("t1_c1").replace("\\", "/")
+    assertTrue(CarbonTablePath.getDictionaryMetaFilePath(identifier.getTablePath(), "t1_c1").replace("\\", "/")
         .equals(CARBON_STORE + "/d1/t1/Metadata/t1_c1.dictmeta"));
-    assertTrue(carbonTablePath.getSortIndexFilePath("t1_c1").replace("\\", "/")
+    assertTrue(CarbonTablePath.getSortIndexFilePath(identifier.getTablePath(),"t1_c1").replace("\\", "/")
         .equals(CARBON_STORE + "/d1/t1/Metadata/t1_c1.sortindex"));
-    assertTrue(carbonTablePath.getCarbonDataFilePath("2", 3, 4L,  0, 0, "999").replace("\\", "/")
+    assertTrue(CarbonTablePath.getCarbonDataFilePath(identifier.getTablePath(), "2", 3, 4L,  0, 0, "999").replace("\\", "/")
         .equals(CARBON_STORE + "/d1/t1/Fact/Part0/Segment_2/part-3-4_batchno0-0-999.carbondata"));
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java
deleted file mode 100644
index 91384c1..0000000
--- a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.util.path;
-
-import java.io.IOException;
-
-import org.junit.Test;
-
-import static junit.framework.TestCase.assertTrue;
-
-/**
- * test shared dictionary paths
- */
-public class CarbonFormatSharedDictionaryTest {
-
-  private final String CARBON_STORE = "/opt/carbonstore";
-
-  /**
-   * test shared dictionary location
-   */
-  @Test public void testSharedDimentionLocation() throws IOException {
-    assertTrue(CarbonSharedDictionaryPath.getDictionaryFilePath(CARBON_STORE, "d1", "shared_c1").replace("\\", "/")
-        .equals(CARBON_STORE + "/d1/SharedDictionary/shared_c1.dict"));
-    assertTrue(CarbonSharedDictionaryPath.getDictionaryMetaFilePath(CARBON_STORE, "d1", "shared_c1").replace("\\", "/")
-        .equals(CARBON_STORE + "/d1/SharedDictionary/shared_c1.dictmeta"));
-    assertTrue(CarbonSharedDictionaryPath.getSortIndexFilePath(CARBON_STORE, "d1", "shared_c1").replace("\\", "/")
-        .equals(CARBON_STORE + "/d1/SharedDictionary/shared_c1.sortindex"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 308d041..d5500e1 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
@@ -44,7 +44,6 @@ import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
 import org.apache.carbondata.core.reader.CarbonDictionaryReaderImpl;
 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;
 import org.apache.carbondata.format.ColumnDictionaryChunkMeta;
 
@@ -72,7 +71,7 @@ public class CarbonDictionaryWriterImplTest {
 
   private String tableName;
 
-  private String carbonStorePath;
+  private String tablePath;
 
   private ColumnIdentifier columnIdentifier;
 
@@ -100,10 +99,10 @@ public class CarbonDictionaryWriterImplTest {
     init();
     this.databaseName = props.getProperty("database", "testSchema");
     this.tableName = props.getProperty("tableName", "carbon");
-    this.carbonStorePath = props.getProperty("storePath", "carbonStore");
+    this.tablePath = props.getProperty("storePath", "carbonStore");
     this.columnIdentifier = new ColumnIdentifier("Name", null, null);
     carbonTableIdentifier = new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    absoluteTableIdentifier = AbsoluteTableIdentifier.from(carbonStorePath, carbonTableIdentifier);
+    absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, carbonTableIdentifier);
     this.dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
             columnIdentifier.getDataType());
@@ -479,8 +478,8 @@ public class CarbonDictionaryWriterImplTest {
    * this method will delete the store path
    */
   private void deleteStorePath() {
-    FileFactory.FileType fileType = FileFactory.getFileType(this.carbonStorePath);
-    CarbonFile carbonFile = FileFactory.getCarbonFile(this.carbonStorePath, fileType);
+    FileFactory.FileType fileType = FileFactory.getFileType(this.tablePath);
+    CarbonFile carbonFile = FileFactory.getCarbonFile(this.tablePath, fileType);
     deleteRecursiveSilent(carbonFile);
   }
 
@@ -528,14 +527,12 @@ public class CarbonDictionaryWriterImplTest {
    * this method will form the dictionary directory paths
    */
   private void initDictionaryDirPaths() throws IOException {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(this.carbonStorePath, carbonTableIdentifier);
-    String dictionaryLocation = carbonTablePath.getMetadataDirectoryPath();
+    String dictionaryLocation = CarbonTablePath.getMetadataPath(tablePath);
     FileFactory.FileType fileType = FileFactory.getFileType(dictionaryLocation);
     if(!FileFactory.isFileExist(dictionaryLocation, fileType)) {
       FileFactory.mkdirs(dictionaryLocation, fileType);
     }
-    this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier.getColumnId());
-    this.dictionaryMetaFilePath = carbonTablePath.getDictionaryMetaFilePath(columnIdentifier.getColumnId());
+    this.dictionaryFilePath = CarbonTablePath.getDictionaryFilePath(tablePath, columnIdentifier.getColumnId());
+    this.dictionaryMetaFilePath = CarbonTablePath.getDictionaryMetaFilePath(tablePath, columnIdentifier.getColumnId());
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonBatchSparkStreamingExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonBatchSparkStreamingExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonBatchSparkStreamingExample.scala
index 6ae87b9..40460c6 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonBatchSparkStreamingExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonBatchSparkStreamingExample.scala
@@ -24,10 +24,6 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{CarbonEnv, SaveMode, SparkSession}
 import org.apache.spark.streaming.{Seconds, StreamingContext, Time}
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
-
 /**
  * This example introduces how to use CarbonData batch load to integrate
  * with Spark Streaming(it's DStream, not Spark Structured Streaming)
@@ -74,7 +70,6 @@ object CarbonBatchSparkStreamingExample {
            | """.stripMargin)
 
       val carbonTable = CarbonEnv.getCarbonTable(Some("default"), streamTableName)(spark)
-      val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
       // batch load
       val path = s"$rootPath/examples/spark2/src/main/resources/streamSample.csv"
       spark.sql(
@@ -88,7 +83,7 @@ object CarbonBatchSparkStreamingExample {
       val serverSocket = new ServerSocket(7071)
       val thread1 = writeSocket(serverSocket)
       val thread2 = showTableCount(spark, streamTableName)
-      val ssc = startStreaming(spark, streamTableName, tablePath, checkpointPath)
+      val ssc = startStreaming(spark, streamTableName, checkpointPath)
       // wait for stop signal to stop Spark Streaming App
       waitForStopSignal(ssc)
       // it need to start Spark Streaming App in main thread
@@ -153,7 +148,7 @@ object CarbonBatchSparkStreamingExample {
   }
 
   def startStreaming(spark: SparkSession, tableName: String,
-      tablePath: CarbonTablePath, checkpointPath: String): StreamingContext = {
+      checkpointPath: String): StreamingContext = {
     var ssc: StreamingContext = null
     try {
       // recommend: the batch interval must set larger, such as 30s, 1min.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonStructuredStreamingExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonStructuredStreamingExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonStructuredStreamingExample.scala
index 247a59b..bca7fa3 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonStructuredStreamingExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonStructuredStreamingExample.scala
@@ -24,8 +24,9 @@ import org.apache.spark.sql.{CarbonEnv, SparkSession}
 import org.apache.spark.sql.streaming.{ProcessingTime, StreamingQuery}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 // scalastyle:off println
 object CarbonStructuredStreamingExample {
@@ -89,7 +90,6 @@ object CarbonStructuredStreamingExample {
       }
 
       val carbonTable = CarbonEnv.getCarbonTable(Some("default"), streamTableName)(spark)
-      val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
       // batch load
       val path = s"$rootPath/examples/spark2/src/main/resources/streamSample.csv"
       spark.sql(
@@ -101,7 +101,7 @@ object CarbonStructuredStreamingExample {
 
       // streaming ingest
       val serverSocket = new ServerSocket(7071)
-      val thread1 = startStreaming(spark, tablePath)
+      val thread1 = startStreaming(spark, carbonTable)
       val thread2 = writeSocket(serverSocket)
       val thread3 = showTableCount(spark, streamTableName)
 
@@ -152,7 +152,7 @@ object CarbonStructuredStreamingExample {
     thread
   }
 
-  def startStreaming(spark: SparkSession, tablePath: CarbonTablePath): Thread = {
+  def startStreaming(spark: SparkSession, carbonTable: CarbonTable): Thread = {
     val thread = new Thread() {
       override def run(): Unit = {
         var qry: StreamingQuery = null
@@ -167,7 +167,8 @@ object CarbonStructuredStreamingExample {
           qry = readSocketDF.writeStream
             .format("carbondata")
             .trigger(ProcessingTime("5 seconds"))
-            .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
+            .option("checkpointLocation",
+              CarbonTablePath.getStreamingCheckpointDir(carbonTable.getTablePath))
             .option("dbName", "default")
             .option("tableName", "stream_table")
             .start()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 b1faa6a..24f5713 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
@@ -69,7 +69,6 @@ import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeConverter;
 import org.apache.carbondata.core.util.DataTypeConverterImpl;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.BlockIndex;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
@@ -248,10 +247,6 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     }
   }
 
-  private static CarbonTablePath getTablePath(AbsoluteTableIdentifier absIdentifier) {
-    return CarbonStorePath.getCarbonTablePath(absIdentifier);
-  }
-
   /**
    * Set list of segments to access
    */
@@ -476,11 +471,10 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     List<InputSplit> splits = new ArrayList<InputSplit>();
     if (streamSegments != null && !streamSegments.isEmpty()) {
 
-      CarbonTablePath tablePath = CarbonStorePath.getCarbonTablePath(identifier);
       long minSize = Math.max(getFormatMinSplitSize(), getMinSplitSize(job));
       long maxSize = getMaxSplitSize(job);
       for (String segmentId : streamSegments) {
-        String segmentDir = tablePath.getSegmentDir(segmentId);
+        String segmentDir = CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId);
         FileFactory.FileType fileType = FileFactory.getFileType(segmentDir);
         if (FileFactory.isFileExist(segmentDir, fileType)) {
           String indexName = CarbonTablePath.getCarbonStreamIndexFileName();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
index 3ef8afc..a4b3be8 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
@@ -37,7 +37,6 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.util.CarbonMetadataUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.FileHeader;
 import org.apache.carbondata.processing.loading.BadRecordsLogger;
@@ -127,9 +126,8 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
     maxCacheSize = hadoopConf.getInt(CarbonStreamOutputFormat.CARBON_STREAM_CACHE_SIZE,
         CarbonStreamOutputFormat.CARBON_STREAM_CACHE_SIZE_DEFAULT);
 
-    CarbonTablePath tablePath =
-        CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
-    segmentDir = tablePath.getSegmentDir(segmentId);
+    segmentDir = CarbonTablePath.getSegmentPath(
+        carbonTable.getAbsoluteTableIdentifier().getTablePath(), segmentId);
     fileName = CarbonTablePath.getCarbonDataFileName(0, taskNo, 0, 0, "0");
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 423bb2a..dfa8dd1 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
@@ -27,7 +27,7 @@ import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverte
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 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.core.util.path.CarbonTablePath;
 
 /**
@@ -37,8 +37,7 @@ public class SchemaReader {
 
   public static CarbonTable readCarbonTableFromStore(AbsoluteTableIdentifier identifier)
       throws IOException {
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(identifier);
-    String schemaFilePath = carbonTablePath.getSchemaFilePath();
+    String schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath());
     if (FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.LOCAL) ||
         FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.HDFS) ||
         FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.S3) ||
@@ -46,7 +45,7 @@ public class SchemaReader {
       String tableName = identifier.getCarbonTableIdentifier().getTableName();
 
       org.apache.carbondata.format.TableInfo tableInfo =
-          CarbonUtil.readSchemaFile(carbonTablePath.getSchemaFilePath());
+          CarbonUtil.readSchemaFile(CarbonTablePath.getSchemaFilePath(identifier.getTablePath()));
       SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
       TableInfo wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
           tableInfo,
@@ -63,22 +62,21 @@ public class SchemaReader {
   /**
    * the method returns the Wrapper TableInfo
    *
-   * @param absoluteTableIdentifier
+   * @param identifier
    * @return
    */
-  public static TableInfo getTableInfo(AbsoluteTableIdentifier absoluteTableIdentifier)
+  public static TableInfo getTableInfo(AbsoluteTableIdentifier identifier)
       throws IOException {
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     org.apache.carbondata.format.TableInfo thriftTableInfo =
-        CarbonUtil.readSchemaFile(carbonTablePath.getSchemaFilePath());
+        CarbonUtil.readSchemaFile(CarbonTablePath.getSchemaFilePath(identifier.getTablePath()));
     ThriftWrapperSchemaConverterImpl thriftWrapperSchemaConverter =
         new ThriftWrapperSchemaConverterImpl();
     CarbonTableIdentifier carbonTableIdentifier =
-        absoluteTableIdentifier.getCarbonTableIdentifier();
+        identifier.getCarbonTableIdentifier();
     return thriftWrapperSchemaConverter.fromExternalToWrapperTableInfo(
         thriftTableInfo,
         carbonTableIdentifier.getDatabaseName(),
         carbonTableIdentifier.getTableName(),
-        absoluteTableIdentifier.getTablePath());
+        identifier.getTablePath());
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 ac17c4e..6365a6a 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
@@ -65,7 +65,6 @@ import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 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;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
@@ -188,11 +187,11 @@ public class StoreCreator {
   }
 
   public static CarbonTable createTable(
-      AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException {
+      AbsoluteTableIdentifier identifier) throws IOException {
     TableInfo tableInfo = new TableInfo();
-    tableInfo.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
+    tableInfo.setDatabaseName(identifier.getCarbonTableIdentifier().getDatabaseName());
     TableSchema tableSchema = new TableSchema();
-    tableSchema.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+    tableSchema.setTableName(identifier.getCarbonTableIdentifier().getTableName());
     List<ColumnSchema> columnSchemas = new ArrayList<ColumnSchema>();
     ArrayList<Encoding> encodings = new ArrayList<>();
     encodings.add(Encoding.DICTIONARY);
@@ -284,13 +283,12 @@ public class StoreCreator {
     tableSchema.setSchemaEvalution(schemaEvol);
     tableSchema.setTableId(UUID.randomUUID().toString());
     tableInfo.setTableUniqueName(
-        absoluteTableIdentifier.getCarbonTableIdentifier().getTableUniqueName()
+        identifier.getCarbonTableIdentifier().getTableUniqueName()
     );
     tableInfo.setLastUpdatedTime(System.currentTimeMillis());
     tableInfo.setFactTable(tableSchema);
-    tableInfo.setTablePath(absoluteTableIdentifier.getTablePath());
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    String schemaFilePath = carbonTablePath.getSchemaFilePath();
+    tableInfo.setTablePath(identifier.getTablePath());
+    String schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath());
     String schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath);
     CarbonMetadata.getInstance().loadTableMetadata(tableInfo);
 
@@ -469,7 +467,7 @@ public class StoreCreator {
     loadMetadataDetails.setLoadStartTime(loadMetadataDetails.getTimeStamp(readCurrentTime()));
     listOfLoadFolderDetails.add(loadMetadataDetails);
 
-    String dataLoadLocation = schema.getCarbonTable().getMetaDataFilepath() + File.separator
+    String dataLoadLocation = schema.getCarbonTable().getMetadataPath() + File.separator
         + CarbonCommonConstants.LOADMETADATA_FILENAME;
 
     DataOutputStream dataOutputStream;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 5a2f831..8f7e88c 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
@@ -25,7 +25,6 @@ import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.service.impl.PathFactory;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
 import org.apache.carbondata.hadoop.CarbonProjection;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
@@ -82,7 +81,8 @@ public class CarbondataRecordSetProvider implements ConnectorRecordSetProvider {
         carbonTableReader.getCarbonCache(carbondataSplit.getSchemaTableName());
     checkNotNull(tableCacheModel, "tableCacheModel should not be null");
     checkNotNull(tableCacheModel.carbonTable, "tableCacheModel.carbonTable should not be null");
-    checkNotNull(tableCacheModel.tableInfo, "tableCacheModel.tableInfo should not be null");
+    checkNotNull(
+        tableCacheModel.carbonTable.getTableInfo(), "tableCacheModel.tableInfo should not be null");
 
     // Build Query Model
     CarbonTable targetTable = tableCacheModel.carbonTable;
@@ -92,8 +92,7 @@ public class CarbondataRecordSetProvider implements ConnectorRecordSetProvider {
     try {
       Configuration conf = new Configuration();
       conf.set(CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS, "");
-      String carbonTablePath = PathFactory.getInstance()
-          .getCarbonTablePath(targetTable.getAbsoluteTableIdentifier(), null).getPath();
+      String carbonTablePath = targetTable.getAbsoluteTableIdentifier().getTablePath();
 
       conf.set(CarbonTableInputFormat.INPUT_DIR, carbonTablePath);
       JobConf jobConf = new JobConf(conf);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java
index 8422c3e..4984406 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java
@@ -17,27 +17,18 @@
 
 package org.apache.carbondata.presto.impl;
 
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.TableInfo;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 /**
- * Caching metadata of CarbonData(e.g. TableIdentifier, TablePath, TableInfo, CarbonTable) in Class CarbonTableReader
+ * Caching metadata of CarbonData in Class CarbonTableReader
  * to speed up query
  */
 public class CarbonTableCacheModel {
 
-  public CarbonTableIdentifier carbonTableIdentifier;
-  public CarbonTablePath carbonTablePath;
-
-  public TableInfo tableInfo;
   public CarbonTable carbonTable;
 
   public boolean isValid() {
-    if (carbonTable != null && carbonTablePath != null && carbonTableIdentifier != null)
-      return true;
-    else return false;
+    return carbonTable != null;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 b0271ef..09389f8 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
@@ -42,7 +42,6 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.reader.ThriftReader;
 import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.service.impl.PathFactory;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
@@ -243,9 +242,10 @@ public class CarbonTableReader {
       updateSchemaList();
     }
     try {
-      if (isKeyExists && !FileFactory
-          .isFileExist(carbonCache.get().get(schemaTableName).carbonTablePath.getSchemaFilePath(),
-              fileType)) {
+      if (isKeyExists
+          && !FileFactory.isFileExist(
+              CarbonTablePath.getSchemaFilePath(
+                  carbonCache.get().get(schemaTableName).carbonTable.getTablePath()), fileType)) {
         removeTableFromCache(schemaTableName);
         throw new TableNotFoundException(schemaTableName);
       }
@@ -255,10 +255,12 @@ public class CarbonTableReader {
 
     if (isKeyExists) {
       CarbonTableCacheModel ctcm = carbonCache.get().get(schemaTableName);
-      if(ctcm != null && ctcm.tableInfo != null) {
-        Long latestTime = FileFactory.getCarbonFile(ctcm.carbonTablePath.getSchemaFilePath())
-            .getLastModifiedTime();
-        Long oldTime = ctcm.tableInfo.getLastUpdatedTime();
+      if(ctcm != null && ctcm.carbonTable.getTableInfo() != null) {
+        Long latestTime = FileFactory.getCarbonFile(
+            CarbonTablePath.getSchemaFilePath(
+                carbonCache.get().get(schemaTableName).carbonTable.getTablePath())
+        ).getLastModifiedTime();
+        Long oldTime = ctcm.carbonTable.getTableInfo().getLastUpdatedTime();
         if (DateUtils.truncate(new Date(latestTime), Calendar.MINUTE)
             .after(DateUtils.truncate(new Date(oldTime), Calendar.MINUTE))) {
           removeTableFromCache(schemaTableName);
@@ -312,21 +314,12 @@ public class CarbonTableReader {
 
       // Step 1: get store path of the table and cache it.
       // create table identifier. the table id is randomly generated.
-      cache.carbonTableIdentifier =
+      CarbonTableIdentifier 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 =
-          AbsoluteTableIdentifier.from(tablePath, cache.carbonTableIdentifier);
-      cache.carbonTablePath =
-          PathFactory.getInstance().getCarbonTablePath(absoluteTableIdentifier, null);
-      // cache the table
-      carbonCache.get().put(table, cache);
+      String tablePath = storePath + "/" + carbonTableIdentifier.getDatabaseName() + "/"
+          + carbonTableIdentifier.getTableName();
 
       //Step 2: read the metadata (tableInfo) of the table.
       ThriftReader.TBaseCreator createTBase = new ThriftReader.TBaseCreator() {
@@ -338,7 +331,7 @@ public class CarbonTableReader {
         }
       };
       ThriftReader thriftReader =
-              new ThriftReader(cache.carbonTablePath.getSchemaFilePath(), createTBase);
+              new ThriftReader(CarbonTablePath.getSchemaFilePath(tablePath), createTBase);
       thriftReader.open();
       org.apache.carbondata.format.TableInfo tableInfo =
               (org.apache.carbondata.format.TableInfo) thriftReader.read();
@@ -355,9 +348,12 @@ public class CarbonTableReader {
       // Step 4: Load metadata info into CarbonMetadata
       CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo);
 
-      cache.tableInfo = wrapperTableInfo;
-      cache.carbonTable = CarbonMetadata.getInstance()
-              .getCarbonTable(cache.carbonTableIdentifier.getTableUniqueName());
+      cache.carbonTable = CarbonMetadata.getInstance().getCarbonTable(
+          table.getSchemaName(), table.getTableName());
+
+      // cache the table
+      carbonCache.get().put(table, cache);
+
       result = cache.carbonTable;
     } catch (Exception ex) {
       throw new RuntimeException(ex);
@@ -372,11 +368,10 @@ public class CarbonTableReader {
     List<CarbonLocalInputSplit> result = new ArrayList<>();
 
     CarbonTable carbonTable = tableCacheModel.carbonTable;
-    TableInfo tableInfo = tableCacheModel.tableInfo;
+    TableInfo tableInfo = tableCacheModel.carbonTable.getTableInfo();
     Configuration config = new Configuration();
     config.set(CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS, "");
-    String carbonTablePath = PathFactory.getInstance()
-        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier(), null).getPath();
+    String carbonTablePath = carbonTable.getAbsoluteTableIdentifier().getTablePath();
     config.set(CarbonTableInputFormat.INPUT_DIR, carbonTablePath);
     config.set(CarbonTableInputFormat.DATABASE_NAME, carbonTable.getDatabaseName());
     config.set(CarbonTableInputFormat.TABLE_NAME, carbonTable.getTableName());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 a41e738..1d7c791 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
@@ -52,7 +52,7 @@ import org.apache.carbondata.core.metadata.schema.{SchemaEvolution, SchemaEvolut
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata,
 CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.writer.sortindex.{CarbonDictionarySortIndexWriter,
 CarbonDictionarySortIndexWriterImpl, CarbonDictionarySortInfo, CarbonDictionarySortInfoPreparator}
@@ -323,10 +323,8 @@ object CarbonDataStoreCreator {
     )
     tableInfo.setLastUpdatedTime(System.currentTimeMillis())
     tableInfo.setFactTable(tableSchema)
-    val carbonTablePath: CarbonTablePath = CarbonStorePath.getCarbonTablePath(
-      absoluteTableIdentifier.getTablePath,
-      absoluteTableIdentifier.getCarbonTableIdentifier)
-    val schemaFilePath: String = carbonTablePath.getSchemaFilePath
+    val schemaFilePath: String = CarbonTablePath.getSchemaFilePath(
+      absoluteTableIdentifier.getTablePath)
     val schemaMetadataPath: String =
       CarbonTablePath.getFolderContainingFile(schemaFilePath)
     CarbonMetadata.getInstance.loadTableMetadata(tableInfo)
@@ -533,7 +531,7 @@ object CarbonDataStoreCreator {
       loadMetadataDetails.setLoadStartTime(
         loadMetadataDetails.getTimeStamp(readCurrentTime()))
       listOfLoadFolderDetails.add(loadMetadataDetails)
-      val dataLoadLocation: String = schema.getCarbonTable.getMetaDataFilepath + File.separator +
+      val dataLoadLocation: String = schema.getCarbonTable.getMetadataPath + File.separator +
                                      CarbonCommonConstants.LOADMETADATA_FILENAME
       val gsonObjectToWrite: Gson = new Gson()
       val writeOperation: AtomicFileOperations = new AtomicFileOperationsImpl(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 cb0d02c..5eb274d 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
@@ -28,7 +28,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 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.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 /**
  * Test Class for AlterTableTestCase to verify all scenerios

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 c84ae6b..a707c60 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
@@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer
 import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterEach
 
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.spark.sql.test.util.QueryTest
@@ -48,13 +48,13 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterEach {
       datbaseName: String,
       tableName: String): Boolean = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(datbaseName, tableName)
-    val partitionPath = CarbonStorePath
-      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir
+    val partitionPath =
+      CarbonTablePath.getPartitionDir(carbonTable.getAbsoluteTableIdentifier.getTablePath)
     val fileType: FileFactory.FileType = FileFactory.getFileType(partitionPath)
     val carbonFile = FileFactory.getCarbonFile(partitionPath, fileType)
     val segments: ArrayBuffer[String] = ArrayBuffer()
     carbonFile.listFiles.foreach { file =>
-      segments += CarbonTablePath.DataPathUtil.getSegmentId(file.getAbsolutePath + "/dummy")
+      segments += CarbonTablePath.DataFileUtil.getSegmentId(file.getAbsolutePath + "/dummy")
     }
     segments.contains(segmentId)
   }
@@ -235,8 +235,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterEach {
     sql("create table stale(a string) stored by 'carbondata'")
     sql("insert into stale values('k')")
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "stale")
-    val tableStatusFile = new CarbonTablePath(null,
-      carbonTable.getTablePath).getTableStatusFilePath
+    val tableStatusFile = CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath)
     FileFactory.getCarbonFile(tableStatusFile).delete()
     sql("insert into stale values('k')")
     checkAnswer(sql("select * from stale"), Row("k"))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 5cc4156..3c2fd71 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
@@ -29,7 +29,7 @@ import org.apache.carbondata.core.datastore.filesystem.CarbonFile
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
   var timeStampPropOrig: String = _
@@ -231,8 +231,7 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("insert overwrite table HiveOverwrite select * from THive")
     checkAnswer(sql("select count(*) from CarbonOverwrite"), sql("select count(*) from HiveOverwrite"))
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "carbonoverwrite")
-    val partitionPath = CarbonStorePath
-      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir
+    val partitionPath = CarbonTablePath.getPartitionDir(carbonTable.getAbsoluteTableIdentifier.getTablePath)
     val folder = new File(partitionPath)
     assert(folder.isDirectory)
     assert(folder.list().length == 1)
@@ -254,8 +253,7 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     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 carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "tcarbonsourceoverwrite")
-    val partitionPath = CarbonStorePath
-      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir
+    val partitionPath = CarbonTablePath.getPartitionDir(carbonTable.getAbsoluteTableIdentifier.getTablePath)
     val folder = new File(partitionPath)
 
     assert(folder.isDirectory)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/concurrent/TestLoadTableConcurrentScenario.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/concurrent/TestLoadTableConcurrentScenario.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/concurrent/TestLoadTableConcurrentScenario.scala
deleted file mode 100644
index 6af28c3..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/concurrent/TestLoadTableConcurrentScenario.scala
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.testsuite.concurrent
-
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
-import org.apache.spark.sql.CarbonEnv
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-class TestLoadTableConcurrentScenario extends QueryTest with BeforeAndAfterAll {
-
-  var carbonTable: CarbonTable = _
-  var metaPath: String = _
-
-  override def beforeAll {
-    sql("use default")
-    sql("drop table if exists drop_concur")
-    sql("drop table if exists rename_concur")
-  }
-
-  test("do not allow drop table when load is in progress") {
-    sql("create table drop_concur(id int, name string) stored by 'carbondata'")
-    sql("insert into drop_concur select 1,'abc'")
-    sql("insert into drop_concur select 1,'abc'")
-    sql("insert into drop_concur select 1,'abc'")
-
-    carbonTable = CarbonEnv.getCarbonTable(Option("default"), "drop_concur")(sqlContext.sparkSession)
-    metaPath = carbonTable.getMetaDataFilepath
-    val listOfLoadFolderDetailsArray = SegmentStatusManager.readLoadMetadata(metaPath)
-    listOfLoadFolderDetailsArray(1).setSegmentStatus(SegmentStatus.INSERT_IN_PROGRESS)
-
-    try {
-      sql("drop table drop_concur")
-    } catch {
-      case ex: Throwable => assert(ex.getMessage.contains("Cannot drop table, load or insert overwrite is in progress"))
-    }
-  }
-
-  test("do not allow rename table when load is in progress") {
-    sql("create table rename_concur(id int, name string) stored by 'carbondata'")
-    sql("insert into rename_concur select 1,'abc'")
-    sql("insert into rename_concur select 1,'abc'")
-
-    carbonTable = CarbonEnv.getCarbonTable(Option("default"), "rename_concur")(sqlContext.sparkSession)
-    metaPath = carbonTable.getMetaDataFilepath
-    val listOfLoadFolderDetailsArray = SegmentStatusManager.readLoadMetadata(metaPath)
-    listOfLoadFolderDetailsArray(1).setSegmentStatus(SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)
-
-    try {
-      sql("alter table rename_concur rename to rename_concur1")
-    } catch {
-      case ex: Throwable => assert(ex.getMessage.contains("alter rename failed, load, insert or insert overwrite " +
-        "is in progress for the table"))
-    }
-  }
-
-  override def afterAll: Unit = {
-    sql("use default")
-    sql("drop table if exists drop_concur")
-    sql("drop table if exists rename_concur")
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala
index 09268b5..8315848 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala
@@ -142,7 +142,7 @@ class TestCreateTableAsSelect extends QueryTest with BeforeAndAfterAll {
     val carbonTable = CarbonEnv.getInstance(Spark2TestQueryExecutor.spark).carbonMetastore
       .lookupRelation(Option("default"), "ctas_tblproperties_test")(Spark2TestQueryExecutor.spark)
       .asInstanceOf[CarbonRelation].carbonTable
-    val metadataFolderPath: CarbonFile = FileFactory.getCarbonFile(carbonTable.getMetaDataFilepath)
+    val metadataFolderPath: CarbonFile = FileFactory.getCarbonFile(carbonTable.getMetadataPath)
     assert(metadataFolderPath.exists())
     val dictFiles: Array[CarbonFile] = metadataFolderPath.listFiles(new CarbonFileFilter {
       override def accept(file: CarbonFile): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 2ef88a4..a7607c3 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
@@ -24,7 +24,7 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.spark.sql.test.util.QueryTest
 
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
@@ -43,9 +43,7 @@ class DataCompactionLockTest extends QueryTest with BeforeAndAfterAll {
         new CarbonTableIdentifier(
           CarbonCommonConstants.DATABASE_DEFAULT_NAME, "compactionlocktesttable", "1")
       )
-  val carbonTablePath: CarbonTablePath = CarbonStorePath
-    .getCarbonTablePath(absoluteTableIdentifier)
-  val dataPath: String = carbonTablePath.getMetadataDirectoryPath
+  val dataPath: String = CarbonTablePath.getMetadataPath(absoluteTableIdentifier.getTablePath)
 
   val carbonLock: ICarbonLock =
     CarbonLockFactory

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 86f974c..5310148 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
@@ -26,7 +26,7 @@ import org.apache.carbondata.core.datastore.TableSegmentUniqueIdentifier
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.hadoop.CacheClient
 import org.apache.spark.sql.test.util.QueryTest
 
@@ -112,11 +112,9 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "ignoremajor"
     )
-    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
 
-    val carbontablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
-      .getMetadataDirectoryPath
-    val segs = SegmentStatusManager.readLoadMetadata(carbontablePath)
+    val carbonTablePath = carbonTable.getMetadataPath
+    val segs = SegmentStatusManager.readLoadMetadata(carbonTablePath)
 
     // status should remain as compacted.
     assertResult(SegmentStatus.COMPACTED)(segs(3).getSegmentStatus)
@@ -134,9 +132,7 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "ignoremajor"
     )
-    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-    val carbontablePath = CarbonStorePath
-      .getCarbonTablePath(absoluteTableIdentifier).getMetadataDirectoryPath
+    val carbontablePath = carbonTable.getMetadataPath
     val segs = SegmentStatusManager.readLoadMetadata(carbontablePath)
 
     // status should remain as compacted for segment 2.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 42ac4df..68a3058 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
@@ -29,7 +29,7 @@ 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.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 class TestBatchSortDataLoad extends QueryTest with BeforeAndAfterAll {
   var filePath: String = _
@@ -193,8 +193,7 @@ class TestBatchSortDataLoad extends QueryTest with BeforeAndAfterAll {
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       tableName
     )
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath(segmentNo)
+    val segmentDir = carbonTable.getSemgentPath(segmentNo)
     new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 db0a62c..b9d8e12 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,7 +22,7 @@ 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.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
@@ -48,8 +48,7 @@ class TestDataLoadWithFileName extends QueryTest with BeforeAndAfterAll {
     sql(s"LOAD DATA LOCAL INPATH '$testData' into table test_table_v3")
     val indexReader = new CarbonIndexFileReader()
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "test_table_v3")
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath("0")
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, "0")
     val carbonIndexPaths = new File(segmentDir)
       .listFiles(new FilenameFilter {
         override def accept(dir: File, name: String): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 fe5e327..5e5eed5 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
@@ -32,8 +32,8 @@ 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
 import org.apache.carbondata.spark.rdd.CarbonScanRDD
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with BeforeAndAfterAll {
   var filePath: String = s"$resourcesPath/globalsort"
@@ -271,8 +271,7 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
     }
     sql(s"LOAD DATA LOCAL INPATH '$inputPath' INTO TABLE carbon_globalsort")
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "carbon_globalsort")
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getSegmentDir("0")
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, "0")
     assertResult(Math.max(4, defaultParallelism) + 1)(new File(segmentDir).listFiles().length)
   }
 
@@ -378,8 +377,7 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
 
   private def getIndexFileCount(tableName: String, segmentNo: String = "0"): Int = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", tableName)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath(segmentNo)
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentNo)
     new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 ed58253..7c82f75 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
@@ -25,7 +25,7 @@ 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, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
@@ -79,8 +79,8 @@ class DataRetentionTestCase extends QueryTest with BeforeAndAfterAll {
       "dataRetentionTable"
     )
     absoluteTableIdentifierForRetention = carbonTable2.getAbsoluteTableIdentifier
-    carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(absoluteTableIdentifierForRetention).getMetadataDirectoryPath
+    carbonTablePath = CarbonTablePath
+      .getMetadataPath(absoluteTableIdentifierForRetention.getTablePath)
     carbonTableStatusLock = CarbonLockFactory
       .getCarbonLockObj(absoluteTableIdentifierForLock, LockUsage.TABLE_STATUS_LOCK)
     carbonDeleteSegmentLock= CarbonLockFactory

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 0a21aed..e5de8da 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,9 +61,7 @@ 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.getCarbonTableIdentifier,
-      carbonTable.getTablePath)
-    val segmentDir = tablePath.getCarbonDataDirectoryPath(segmentId)
+    val segmentDir = carbonTable.getSemgentPath(segmentId)
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {
       override def accept(file: CarbonFile): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
index 5427981..877d026 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
@@ -51,9 +51,7 @@ class StandardPartitionTableCleanTestCase extends QueryTest with BeforeAndAfterA
 
   def validateDataFiles(tableUniqueName: String, segmentId: String, partitions: Int, partitionMapFiles: Int): Unit = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
-    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
-      carbonTable.getTablePath)
-    val segmentDir = tablePath.getCarbonDataDirectoryPath(segmentId)
+    val segmentDir = carbonTable.getSemgentPath(segmentId)
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {
       override def accept(file: CarbonFile): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCompactionTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCompactionTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCompactionTestCase.scala
index f4b6e0e..40da6f1 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCompactionTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCompactionTestCase.scala
@@ -51,9 +51,7 @@ class StandardPartitionTableCompactionTestCase extends QueryTest with BeforeAndA
 
   def validateDataFiles(tableUniqueName: String, segmentId: String, partitions: Int): Unit = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
-    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
-      carbonTable.getTablePath)
-    val segmentDir = tablePath.getCarbonDataDirectoryPath(segmentId)
+    val segmentDir = carbonTable.getSemgentPath(segmentId)
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {
       override def accept(file: CarbonFile): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
index 7fcfb29..6358c83 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
@@ -66,9 +66,7 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
 
   def validateDataFiles(tableUniqueName: String, segmentId: String, partitions: Int): Unit = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
-    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
-      carbonTable.getTablePath)
-    val segmentDir = tablePath.getCarbonDataDirectoryPath(segmentId)
+    val segmentDir = carbonTable.getSemgentPath(segmentId)
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {
       override def accept(file: CarbonFile): Boolean = {
@@ -333,9 +331,7 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE mergeindexpartitionthree OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
 
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_mergeindexpartitionthree")
-    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
-      carbonTable.getTablePath)
-    val segmentDir = tablePath.getCarbonDataDirectoryPath("0")
+    val segmentDir = carbonTable.getSemgentPath("0")
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val files = carbonFile.listFiles(new CarbonFileFilter {
       override def accept(file: CarbonFile): Boolean = CarbonTablePath.isCarbonIndexFile(file.getName)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 c02ba0a..b98bddf 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
@@ -173,7 +173,7 @@ object CarbonStore {
     LOGGER.audit(s"Delete segment by Id request has been received for $dbName.$tableName")
     validateLoadIds(loadids)
 
-    val path = carbonTable.getMetaDataFilepath
+    val path = carbonTable.getMetadataPath
 
     try {
       val invalidLoadIds = SegmentStatusManager.updateDeletionStatus(
@@ -199,7 +199,7 @@ object CarbonStore {
     LOGGER.audit(s"Delete segment by Id request has been received for $dbName.$tableName")
 
     val time = validateTimeFormat(timestamp)
-    val path = carbonTable.getMetaDataFilepath
+    val path = carbonTable.getMetadataPath
 
     try {
       val invalidLoadTimestamps =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala
index 32d121e..3dd9903 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala
@@ -30,7 +30,7 @@ trait ColumnValidator {
  */
 trait DictionaryDetailService {
   def getDictionaryDetail(dictFolderPath: String, primDimensions: Array[CarbonDimension],
-      table: CarbonTableIdentifier, storePath: String): DictionaryDetail
+      tablePath: String): DictionaryDetail
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/integration/spark-common/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala
index e861a8c..dbf47ab 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala
@@ -23,12 +23,11 @@ import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFi
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 class DictionaryDetailHelper extends DictionaryDetailService {
-  def getDictionaryDetail(dictfolderPath: String, primDimensions: Array[CarbonDimension],
-      table: CarbonTableIdentifier, storePath: String): DictionaryDetail = {
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, table)
+  override def getDictionaryDetail(dictfolderPath: String, primDimensions: Array[CarbonDimension],
+      tablePath: String): DictionaryDetail = {
     val dictFilePaths = new Array[String](primDimensions.length)
     val dictFileExists = new Array[Boolean](primDimensions.length)
     val columnIdentifier = new Array[ColumnIdentifier](primDimensions.length)
@@ -50,7 +49,7 @@ class DictionaryDetailHelper extends DictionaryDetailService {
     // 3 lookup fileNamesMap, if file name is in fileNamesMap, file is exists, or not.
     primDimensions.zipWithIndex.foreach { f =>
       columnIdentifier(f._2) = f._1.getColumnIdentifier
-      dictFilePaths(f._2) = carbonTablePath.getDictionaryFilePath(f._1.getColumnId)
+      dictFilePaths(f._2) = CarbonTablePath.getDictionaryFilePath(tablePath, f._1.getColumnId)
       dictFileExists(f._2) =
         fileNamesMap.get(CarbonTablePath.getDictionaryFileName(f._1.getColumnId)) match {
           case None => false

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 f2f4ecd..56a66b9 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
@@ -26,7 +26,7 @@ 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
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.spark.util.GlobalDictionaryUtil
 
 /**
@@ -49,7 +49,7 @@ class AddColumnPartition(rddId: Int, idx: Int, schema: ColumnSchema) extends Par
  */
 class AlterTableAddColumnRDD[K, V](sc: SparkContext,
     @transient newColumns: Seq[ColumnSchema],
-    absoluteTableIdentifier: AbsoluteTableIdentifier)
+    identifier: AbsoluteTableIdentifier)
   extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
 
   val lockType: String = CarbonProperties.getInstance.getProperty(CarbonCommonConstants.LOCK_TYPE,
@@ -70,8 +70,6 @@ class AlterTableAddColumnRDD[K, V](sc: SparkContext,
         // create dictionary file if it is a dictionary column
         if (columnSchema.hasEncoding(Encoding.DICTIONARY) &&
             !columnSchema.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          val carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier)
           var rawData: String = null
           if (null != columnSchema.getDefaultValue) {
             rawData = new String(columnSchema.getDefaultValue,
@@ -79,16 +77,15 @@ class AlterTableAddColumnRDD[K, V](sc: SparkContext,
           }
           CarbonProperties.getInstance.addProperty(CarbonCommonConstants.LOCK_TYPE, lockType)
           // Create table and metadata folders if not exist
-          val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
+          val metadataDirectoryPath = CarbonTablePath.getMetadataPath(identifier.getTablePath)
           val fileType = FileFactory.getFileType(metadataDirectoryPath)
           if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
             FileFactory.mkdirs(metadataDirectoryPath, fileType)
           }
-          GlobalDictionaryUtil
-            .loadDefaultDictionaryValueForNewColumn(carbonTablePath,
-              columnSchema,
-              absoluteTableIdentifier,
-              rawData)
+          GlobalDictionaryUtil.loadDefaultDictionaryValueForNewColumn(
+            columnSchema,
+            identifier,
+            rawData)
         }
       } catch {
         case ex: Exception =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3e99681/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 db29532..7acf4e2 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
@@ -40,10 +40,8 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
 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
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, CarbonUtil}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.processing.loading.exception.NoRetryException
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
@@ -348,10 +346,6 @@ class CarbonGlobalDictionaryGenerateRDD(
         model.table,
         model.columnIdentifier(split.index),
         model.columnIdentifier(split.index).getDataType)
-      val pathService: PathService = CarbonCommonFactory.getPathService
-      val carbonTablePath: CarbonTablePath =
-        pathService
-          .getCarbonTablePath(model.table, dictionaryColumnUniqueIdentifier)
       if (StringUtils.isNotBlank(model.hdfsTempLocation)) {
         CarbonProperties.getInstance.addProperty(CarbonCommonConstants.HDFS_TEMP_LOCATION,
           model.hdfsTempLocation)