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:47 UTC

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

[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);