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

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

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)