You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by gv...@apache.org on 2017/12/07 17:09:58 UTC

[2/2] carbondata git commit: [CARBONDATA-1870] Add dictionary path support to carbondata

[CARBONDATA-1870] Add dictionary path support to carbondata

This closes #1617


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

Branch: refs/heads/master
Commit: 29dc30280d37e39306006c288546206db06d1bf4
Parents: 2304303
Author: manishgupta88 <to...@gmail.com>
Authored: Tue Dec 5 20:39:43 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Thu Dec 7 22:39:39 2017 +0530

----------------------------------------------------------------------
 .../dictionary/AbstractDictionaryCache.java     |  9 +-
 .../DictionaryColumnUniqueIdentifier.java       | 95 +++++++++++++++-----
 .../core/constants/CarbonCommonConstants.java   |  4 +
 .../IncrementalColumnDictionaryGenerator.java   | 11 +--
 .../CarbonDictionaryMetadataReaderImpl.java     | 11 +--
 .../core/reader/CarbonDictionaryReaderImpl.java | 10 +--
 .../CarbonDictionarySortIndexReaderImpl.java    | 16 +---
 .../core/scan/executor/util/QueryUtil.java      | 13 +--
 .../carbondata/core/scan/filter/FilterUtil.java | 18 ++--
 .../apache/carbondata/core/util/CarbonUtil.java | 13 +--
 .../core/util/path/CarbonTablePath.java         | 40 ++++++++-
 .../core/writer/CarbonDictionaryWriterImpl.java | 13 +--
 .../CarbonDictionarySortIndexWriterImpl.java    | 26 ++----
 .../dictionary/AbstractDictionaryCacheTest.java |  6 +-
 .../DictionaryCacheLoaderImplTest.java          |  3 +-
 .../dictionary/ForwardDictionaryCacheTest.java  |  3 +-
 .../reader/CarbonDictionaryReaderImplTest.java  |  3 +-
 ...CarbonDictionarySortIndexReaderImplTest.java |  3 +-
 .../writer/CarbonDictionaryWriterImplTest.java  |  3 +-
 ...CarbonDictionarySortIndexWriterImplTest.java |  3 +-
 .../carbondata/hadoop/CarbonRecordReader.java   |  3 +-
 .../hadoop/readsupport/CarbonReadSupport.java   |  7 +-
 .../impl/DictionaryDecodeReadSupport.java       | 15 ++--
 .../readsupport/impl/RawDataReadSupport.java    |  5 +-
 .../streaming/CarbonStreamInputFormat.java      |  7 +-
 .../hadoop/test/util/StoreCreator.java          |  8 +-
 .../hive/CarbonDictionaryDecodeReadSupport.java | 13 +--
 .../carbondata/hive/CarbonHiveRecordReader.java |  2 +-
 .../carbondata/presto/CarbondataRecordSet.java  |  2 +-
 .../CarbonDictionaryDecodeReadSupport.scala     | 12 +--
 .../presto/util/CarbonDataStoreCreator.scala    | 11 +--
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |  3 +-
 .../spark/util/GlobalDictionaryUtil.scala       |  9 +-
 .../readsupport/SparkRowReadSupportImpl.java    |  4 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     | 54 ++++++-----
 .../spark/util/DictionaryTestCaseUtil.scala     |  4 +-
 .../processing/datatypes/PrimitiveDataType.java |  4 +-
 .../converter/impl/FieldEncoderFactory.java     |  6 +-
 .../processing/util/CarbonLoaderUtil.java       |  3 +-
 .../carbondata/processing/StoreCreator.java     |  7 +-
 40 files changed, 244 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
index d527e68..598d00e 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
@@ -30,10 +30,8 @@ import org.apache.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReader;
 import org.apache.carbondata.core.service.CarbonCommonFactory;
 import org.apache.carbondata.core.service.DictionaryService;
-import org.apache.carbondata.core.service.PathService;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.ObjectSizeCalculator;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 /**
  * Abstract class which implements methods common to reverse and forward dictionary cache
@@ -141,12 +139,7 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
    */
   private CarbonFile getDictionaryMetaCarbonFile(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) throws IOException {
-    PathService pathService = CarbonCommonFactory.getPathService();
-    CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
-            dictionaryColumnUniqueIdentifier);
-    String dictionaryFilePath = carbonTablePath.getDictionaryMetaFilePath(
-        dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
+    String dictionaryFilePath = dictionaryColumnUniqueIdentifier.getDictionaryFilePath();
     FileFactory.FileType fileType = FileFactory.getFileType(dictionaryFilePath);
     CarbonFile dictFile = FileFactory.getCarbonFile(dictionaryFilePath, fileType);
     // When rename table triggered parallely with select query, dictionary files may not exist

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/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 1c733b4..f5652ac 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,6 +22,7 @@ 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;
 
 /**
@@ -34,7 +35,7 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
   /**
    * table fully qualified name
    */
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
+  private AbsoluteTableIdentifier dictionarySourceAbsoluteTableIdentifier;
 
   /**
    * unique column id
@@ -45,40 +46,53 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
 
   private DataType dataType;
 
+  private String dictionaryLocation;
+
   /**
    * Will be used in case of reverse dictionary cache which will be used
    * in case of data loading.
    *
-   * @param absoluteTableIdentifier
+   * @param dictionarySourceAbsoluteTableIdentifier
    * @param columnIdentifier
    */
-  public DictionaryColumnUniqueIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier,
+  public DictionaryColumnUniqueIdentifier(
+      AbsoluteTableIdentifier dictionarySourceAbsoluteTableIdentifier,
       ColumnIdentifier columnIdentifier) {
-    if (absoluteTableIdentifier == null) {
+    if (dictionarySourceAbsoluteTableIdentifier == null) {
       throw new IllegalArgumentException("carbonTableIdentifier is null");
     }
     if (columnIdentifier == null) {
       throw new IllegalArgumentException("columnIdentifier is null");
     }
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
+    this.dictionarySourceAbsoluteTableIdentifier = dictionarySourceAbsoluteTableIdentifier;
     this.columnIdentifier = columnIdentifier;
     this.dataType = columnIdentifier.getDataType();
+    this.dictionaryLocation =
+        CarbonStorePath.getCarbonTablePath(dictionarySourceAbsoluteTableIdentifier)
+            .getMetadataDirectoryPath();
   }
 
   /**
    * Will be used in case of forward dictionary cache in case
    * of query execution.
    *
-   * @param absoluteTableIdentifier
+   * @param dictionarySourceAbsoluteTableIdentifier
    * @param columnIdentifier
    * @param dataType
    */
-  public DictionaryColumnUniqueIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier,
-      ColumnIdentifier columnIdentifier, DataType dataType, CarbonTablePath carbonTablePath) {
-    this(absoluteTableIdentifier, columnIdentifier);
+  public DictionaryColumnUniqueIdentifier(
+      AbsoluteTableIdentifier dictionarySourceAbsoluteTableIdentifier,
+      ColumnIdentifier columnIdentifier, DataType dataType) {
+    this(dictionarySourceAbsoluteTableIdentifier, columnIdentifier);
     this.dataType = dataType;
-    if (null != carbonTablePath) {
-      this.carbonTablePath = carbonTablePath;
+  }
+
+  public DictionaryColumnUniqueIdentifier(
+      AbsoluteTableIdentifier dictionarySourceAbsoluteTableIdentifier,
+      ColumnIdentifier columnIdentifier, DataType dataType, String dictionaryLocation) {
+    this(dictionarySourceAbsoluteTableIdentifier, columnIdentifier, dataType);
+    if (null != dictionaryLocation) {
+      this.dictionaryLocation = dictionaryLocation;
     }
   }
 
@@ -86,14 +100,10 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
     return dataType;
   }
 
-  /**
-   * @return table identifier
-   */
-  public AbsoluteTableIdentifier getAbsoluteCarbonTableIdentifier() {
-    return absoluteTableIdentifier;
-  }
-
   public CarbonTablePath getCarbonTablePath() {
+    if (null == carbonTablePath) {
+      carbonTablePath = CarbonStorePath.getCarbonTablePath(dictionarySourceAbsoluteTableIdentifier);
+    }
     return carbonTablePath;
   }
 
@@ -105,18 +115,57 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
   }
 
   /**
+   * @return dictionary file path
+   */
+  public String getDictionaryFilePath() {
+    return getCarbonTablePath()
+        .getDictionaryFilePath(dictionaryLocation, columnIdentifier.getColumnId());
+  }
+
+  /**
+   * @return dictionary metadata file path
+   */
+  public String getDictionaryMetaFilePath() {
+    return getCarbonTablePath()
+        .getDictionaryMetaFilePath(dictionaryLocation, columnIdentifier.getColumnId());
+  }
+
+  /**
+   * @return sort index file path
+   */
+  public String getSortIndexFilePath() {
+    return getCarbonTablePath()
+        .getSortIndexFilePath(dictionaryLocation, columnIdentifier.getColumnId());
+  }
+
+  /**
+   * @param offset
+   * @return sort index file path with given offset
+   */
+  public String getSortIndexFilePath(long offset) {
+    return getCarbonTablePath()
+        .getSortIndexFilePath(dictionaryLocation, columnIdentifier.getColumnId(), offset);
+  }
+
+  /**
    * overridden equals method
    *
    * @param other
    * @return
    */
   @Override public boolean equals(Object other) {
-    if (this == other) return true;
-    if (other == null || getClass() != other.getClass()) return false;
+    if (this == other) {
+      return true;
+    }
+    if (other == null || getClass() != other.getClass()) {
+      return false;
+    }
     DictionaryColumnUniqueIdentifier that = (DictionaryColumnUniqueIdentifier) other;
-    if (!absoluteTableIdentifier.equals(that.absoluteTableIdentifier)) return false;
+    if (!dictionarySourceAbsoluteTableIdentifier
+        .equals(that.dictionarySourceAbsoluteTableIdentifier)) {
+      return false;
+    }
     return columnIdentifier.equals(that.columnIdentifier);
-
   }
 
   /**
@@ -125,7 +174,7 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
    * @return
    */
   @Override public int hashCode() {
-    int result = absoluteTableIdentifier.hashCode();
+    int result = dictionarySourceAbsoluteTableIdentifier.hashCode();
     result = 31 * result + columnIdentifier.hashCode();
     return result;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 5fb08a3..4ab9919 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -859,6 +859,10 @@ public final class CarbonCommonConstants {
   public static final String COLUMN_GROUPS = "column_groups";
   public static final String DICTIONARY_EXCLUDE = "dictionary_exclude";
   public static final String DICTIONARY_INCLUDE = "dictionary_include";
+  /**
+   * key for dictionary path
+   */
+  public static final String DICTIONARY_PATH = "dictionary_path";
   public static final String SORT_COLUMNS = "sort_columns";
   public static final String PARTITION_TYPE = "partition_type";
   public static final String NUM_PARTITIONS = "num_partitions";

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
index a129671..5eeeae4 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
@@ -41,8 +41,6 @@ import org.apache.carbondata.core.service.CarbonCommonFactory;
 import org.apache.carbondata.core.service.DictionaryService;
 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.core.writer.CarbonDictionaryWriter;
 import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
 import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfo;
@@ -116,15 +114,13 @@ public class IncrementalColumnDictionaryGenerator implements BiDictionary<Intege
 
   @Override public void writeDictionaryData() throws IOException {
     // initialize params
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
     AbsoluteTableIdentifier absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier();
     ColumnIdentifier columnIdentifier = dimension.getColumnIdentifier();
     DictionaryService dictionaryService = CarbonCommonFactory.getDictionaryService();
     // create dictionary cache from dictionary File
     DictionaryColumnUniqueIdentifier identifier =
-            new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-                    columnIdentifier.getDataType(), carbonTablePath);
+        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
+            columnIdentifier.getDataType());
     Boolean isDictExists = CarbonUtil.isFileExistsForGivenColumn(identifier);
     Dictionary dictionary = null;
     long t1 = System.currentTimeMillis();
@@ -216,10 +212,9 @@ public class IncrementalColumnDictionaryGenerator implements BiDictionary<Intege
                               AbsoluteTableIdentifier absoluteTableIdentifier,
                               ColumnIdentifier columnIdentifier) throws IOException {
     CarbonDictionarySortIndexWriter carbonDictionarySortIndexWriter = null;
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType(), carbonTablePath);
+            columnIdentifier.getDataType());
     try {
       CarbonDictionarySortInfoPreparator preparator = new CarbonDictionarySortInfoPreparator();
       CarbonDictionarySortInfo dictionarySortInfo =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
index fa3c1eb..6439b36 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
@@ -23,9 +23,6 @@ import java.util.List;
 
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.service.CarbonCommonFactory;
-import org.apache.carbondata.core.service.PathService;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.ColumnDictionaryChunkMeta;
 
 import org.apache.thrift.TBase;
@@ -153,12 +150,8 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
    * This method will form the path for dictionary metadata file for a given column
    */
   protected void initFileLocation() {
-    PathService pathService = CarbonCommonFactory.getPathService();
-    CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
-            dictionaryColumnUniqueIdentifier);
-    this.columnDictionaryMetadataFilePath = carbonTablePath.getDictionaryMetaFilePath(
-        dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
+    this.columnDictionaryMetadataFilePath =
+        dictionaryColumnUniqueIdentifier.getDictionaryMetaFilePath();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
index 539ee16..c66b9fb 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
@@ -25,9 +25,6 @@ import java.util.List;
 
 import org.apache.carbondata.core.cache.dictionary.ColumnDictionaryChunkIterator;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.service.CarbonCommonFactory;
-import org.apache.carbondata.core.service.PathService;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.ColumnDictionaryChunk;
 
 import org.apache.thrift.TBase;
@@ -200,12 +197,7 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
    * This method will form the path for dictionary file for a given column
    */
   protected void initFileLocation() {
-    PathService pathService = CarbonCommonFactory.getPathService();
-    CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
-            dictionaryColumnUniqueIdentifier);
-    this.columnDictionaryFilePath = carbonTablePath.getDictionaryFilePath(
-        dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
+    this.columnDictionaryFilePath = dictionaryColumnUniqueIdentifier.getDictionaryFilePath();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
index 12426f4..439a93e 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
@@ -27,9 +27,6 @@ import org.apache.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReader;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
 import org.apache.carbondata.core.reader.ThriftReader;
-import org.apache.carbondata.core.service.CarbonCommonFactory;
-import org.apache.carbondata.core.service.PathService;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.ColumnSortInfo;
 
 import org.apache.thrift.TBase;
@@ -135,24 +132,17 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
   }
 
   protected void initPath() {
-    PathService pathService = CarbonCommonFactory.getPathService();
-    CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
-            dictionaryColumnUniqueIdentifier);
     try {
       CarbonDictionaryColumnMetaChunk chunkMetaObjectForLastSegmentEntry =
           getChunkMetaObjectForLastSegmentEntry();
       long dictOffset = chunkMetaObjectForLastSegmentEntry.getEnd_offset();
-      this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(
-          dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(), dictOffset);
+      this.sortIndexFilePath = dictionaryColumnUniqueIdentifier.getSortIndexFilePath(dictOffset);
       if (!FileFactory
           .isFileExist(this.sortIndexFilePath, FileFactory.getFileType(this.sortIndexFilePath))) {
-        this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(
-            dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
+        this.sortIndexFilePath = dictionaryColumnUniqueIdentifier.getSortIndexFilePath();
       }
     } catch (IOException e) {
-      this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(
-          dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
+      this.sortIndexFilePath = dictionaryColumnUniqueIdentifier.getSortIndexFilePath();
     }
 
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/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 b5c56dc..b33ab8d 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
@@ -385,23 +385,24 @@ public class QueryUtil {
       CarbonDimension dimension = CarbonMetadata.getInstance()
           .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnId);
       if (dimension != null) {
-        AbsoluteTableIdentifier newCarbonTableIdentifier;
+        AbsoluteTableIdentifier dictionarySourceAbsoluteTableIdentifier;
         ColumnIdentifier columnIdentifier;
         if (null != dimension.getColumnSchema().getParentColumnTableRelations() && !dimension
             .getColumnSchema().getParentColumnTableRelations().isEmpty()) {
-          newCarbonTableIdentifier = getTableIdentifierForColumn(dimension,
+          dictionarySourceAbsoluteTableIdentifier = getTableIdentifierForColumn(dimension,
               carbonTable.getAbsoluteTableIdentifier());
           columnIdentifier = new ColumnIdentifier(
               dimension.getColumnSchema().getParentColumnTableRelations().get(0).getColumnId(),
               dimension.getColumnProperties(), dimension.getDataType());
         } else {
-          newCarbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier();
+          dictionarySourceAbsoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier();
           columnIdentifier = dimension.getColumnIdentifier();
         }
+        String dictionaryPath = carbonTable.getTableInfo().getFactTable().getTableProperties()
+            .get(CarbonCommonConstants.DICTIONARY_PATH);
         dictionaryColumnUniqueIdentifiers.add(
-            new DictionaryColumnUniqueIdentifier(newCarbonTableIdentifier, columnIdentifier,
-                dimension.getDataType(),
-                CarbonStorePath.getCarbonTablePath(newCarbonTableIdentifier)));
+            new DictionaryColumnUniqueIdentifier(dictionarySourceAbsoluteTableIdentifier,
+                columnIdentifier, dimension.getDataType(), dictionaryPath));
       }
     }
     return dictionaryColumnUniqueIdentifiers;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
index 16b3b1f..a8fa8ec 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
@@ -101,8 +101,6 @@ import org.apache.carbondata.core.util.DataTypeConverterImpl;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.comparator.Comparator;
 import org.apache.carbondata.core.util.comparator.SerializableComparator;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.roaringbitmap.RoaringBitmap;
@@ -1240,18 +1238,18 @@ public final class FilterUtil {
    * @return
    */
   public static Dictionary getForwardDictionaryCache(
-      AbsoluteTableIdentifier absoluteTableIdentifier,
+      AbsoluteTableIdentifier dictionarySourceAbsoluteTableIdentifier,
       CarbonDimension carbonDimension, TableProvider tableProvider) throws IOException {
-    CarbonTablePath carbonTablePath = null;
+    String dictionaryPath = null;
     if (null != tableProvider) {
-      CarbonTable carbonTable =
-          tableProvider.getCarbonTable(absoluteTableIdentifier.getCarbonTableIdentifier());
-      carbonTablePath =
-          CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
+      CarbonTable carbonTable = tableProvider
+          .getCarbonTable(dictionarySourceAbsoluteTableIdentifier.getCarbonTableIdentifier());
+      dictionaryPath = carbonTable.getTableInfo().getFactTable().getTableProperties()
+          .get(CarbonCommonConstants.DICTIONARY_PATH);
     }
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
-            carbonDimension.getColumnIdentifier(), carbonDimension.getDataType(), carbonTablePath);
+        new DictionaryColumnUniqueIdentifier(dictionarySourceAbsoluteTableIdentifier,
+            carbonDimension.getColumnIdentifier(), carbonDimension.getDataType(), dictionaryPath);
     CacheProvider cacheProvider = CacheProvider.getInstance();
     Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache =
         cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/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 1e8bc1f..ab85684 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
@@ -76,8 +76,6 @@ import org.apache.carbondata.core.mutate.UpdateVO;
 import org.apache.carbondata.core.reader.ThriftReader;
 import org.apache.carbondata.core.reader.ThriftReader.TBaseCreator;
 import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.service.CarbonCommonFactory;
-import org.apache.carbondata.core.service.PathService;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
@@ -1636,14 +1634,9 @@ public final class CarbonUtil {
    */
   public static boolean isFileExistsForGivenColumn(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    PathService pathService = CarbonCommonFactory.getPathService();
-    CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
-            dictionaryColumnUniqueIdentifier);
-    String dictionaryFilePath = carbonTablePath.getDictionaryFilePath(
-        dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
-    String dictionaryMetadataFilePath = carbonTablePath.getDictionaryMetaFilePath(
-        dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
+    String dictionaryFilePath = dictionaryColumnUniqueIdentifier.getDictionaryFilePath();
+    String dictionaryMetadataFilePath =
+        dictionaryColumnUniqueIdentifier.getDictionaryMetaFilePath();
     // check if both dictionary and its metadata file exists for a given column
     return isFileExists(dictionaryFilePath) && isFileExists(dictionaryMetadataFilePath);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/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 172b71d..0f85b64 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
@@ -139,6 +139,15 @@ public class CarbonTablePath extends Path {
   }
 
   /**
+   * @param dictionaryPath
+   * @param columnId unique column identifier
+   * @return absolute path of dictionary file
+   */
+  public String getDictionaryFilePath(String dictionaryPath, String columnId) {
+    return dictionaryPath + File.separator + getDictionaryFileName(columnId);
+  }
+
+  /**
    * This method will return the metadata directory location for a table
    *
    * @return
@@ -163,6 +172,15 @@ public class CarbonTablePath extends Path {
   }
 
   /**
+   * @param dictionaryPath
+   * @param columnId unique column identifier
+   * @return absolute path of dictionary file
+   */
+  public String getDictionaryMetaFilePath(String dictionaryPath, String columnId) {
+    return dictionaryPath + File.separator + columnId + DICTIONARY_META_EXT;
+  }
+
+  /**
    * @param columnId unique column identifier
    * @return absolute path of sort index file
    */
@@ -171,6 +189,15 @@ public class CarbonTablePath extends Path {
   }
 
   /**
+   * @param dictionaryPath
+   * @param columnId unique column identifier
+   * @return absolute path of dictionary file
+   */
+  public String getSortIndexFilePath(String dictionaryPath, String columnId) {
+    return dictionaryPath + File.separator + columnId + SORT_INDEX_EXT;
+  }
+
+  /**
    *
    * @param columnId
    * @param dictOffset
@@ -181,6 +208,16 @@ public class CarbonTablePath extends Path {
   }
 
   /**
+   * @param dictionaryPath
+   * @param columnId unique column identifier
+   * @param dictOffset
+   * @return absolute path of dictionary file
+   */
+  public String getSortIndexFilePath(String dictionaryPath, String columnId, long dictOffset) {
+    return dictionaryPath + File.separator + columnId + "_" + dictOffset + SORT_INDEX_EXT;
+  }
+
+  /**
    * @return absolute path of schema file
    */
   public String getSchemaFilePath() {
@@ -526,7 +563,8 @@ public class CarbonTablePath extends Path {
    * @param columnUniqueId   columnunique id
    * @return sort index carbon files
    */
-  public CarbonFile[] getSortIndexFiles(CarbonFile sortIndexDir, final String columnUniqueId) {
+  public static CarbonFile[] getSortIndexFiles(CarbonFile sortIndexDir,
+      final String columnUniqueId) {
     return sortIndexDir.listFiles(new CarbonFileFilter() {
       @Override public boolean accept(CarbonFile file) {
         return file.getName().startsWith(columnUniqueId) && file.getName().endsWith(SORT_INDEX_EXT);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
index 6e867bb..4014cb4 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
@@ -32,10 +32,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReader;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
-import org.apache.carbondata.core.service.CarbonCommonFactory;
-import org.apache.carbondata.core.service.PathService;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.util.path.HDFSLeaseUtils;
 import org.apache.carbondata.format.ColumnDictionaryChunk;
 import org.apache.carbondata.format.ColumnDictionaryChunkMeta;
@@ -238,14 +235,8 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
   }
 
   protected void initPaths() {
-    PathService pathService = CarbonCommonFactory.getPathService();
-    CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
-            dictionaryColumnUniqueIdentifier);
-    this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(
-        dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
-    this.dictionaryMetaFilePath = carbonTablePath.getDictionaryMetaFilePath(
-        dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
+    this.dictionaryFilePath = dictionaryColumnUniqueIdentifier.getDictionaryFilePath();
+    this.dictionaryMetaFilePath = dictionaryColumnUniqueIdentifier.getDictionaryMetaFilePath();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
index 840b25d..f25081d 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
@@ -27,8 +27,6 @@ import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.service.CarbonCommonFactory;
-import org.apache.carbondata.core.service.PathService;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -134,28 +132,20 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
   }
 
   protected void initPath() {
-    PathService pathService = CarbonCommonFactory.getPathService();
-    CarbonTablePath carbonTablePath = pathService
-        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getAbsoluteCarbonTableIdentifier(),
-            dictionaryColumnUniqueIdentifier);
-    String dictionaryPath = carbonTablePath.getDictionaryFilePath(
-        dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
-    long dictOffset = CarbonUtil.getFileSize(dictionaryPath);
-    this.sortIndexFilePath = carbonTablePath
-        .getSortIndexFilePath(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
-            dictOffset);
-    cleanUpOldSortIndex(carbonTablePath, dictionaryPath);
+    String dictionaryFilePath = dictionaryColumnUniqueIdentifier.getDictionaryFilePath();
+    long dictOffset = CarbonUtil.getFileSize(dictionaryFilePath);
+    this.sortIndexFilePath = dictionaryColumnUniqueIdentifier.getSortIndexFilePath(dictOffset);
+    cleanUpOldSortIndex(dictionaryFilePath);
   }
 
   /**
    * It cleans up old unused sortindex file
    *
-   * @param carbonTablePath
+   * @param dictPath
    */
-  protected void cleanUpOldSortIndex(CarbonTablePath carbonTablePath, String dictPath) {
-    CarbonFile dictFile =
-        FileFactory.getCarbonFile(dictPath, FileFactory.getFileType(dictPath));
-    CarbonFile[] files = carbonTablePath.getSortIndexFiles(dictFile.getParentFile(),
+  protected void cleanUpOldSortIndex(String dictPath) {
+    CarbonFile dictFile = FileFactory.getCarbonFile(dictPath, FileFactory.getFileType(dictPath));
+    CarbonFile[] files = CarbonTablePath.getSortIndexFiles(dictFile.getParentFile(),
         dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
     int maxTime;
     try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/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 16c09fd..31e44a2 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
@@ -108,8 +108,7 @@ public class AbstractDictionaryCacheTest {
       String columnId) {
 	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataTypes.STRING);
     return new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-        DataTypes.STRING,
-        CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
+        DataTypes.STRING);
   }
 
   /**
@@ -132,8 +131,7 @@ public class AbstractDictionaryCacheTest {
 	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType(),
-            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
+            columnIdentifier.getDataType());
     CarbonDictionaryWriter carbonDictionaryWriter =
         new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
     CarbonTablePath carbonTablePath =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
index 7b4a076..83e9217 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
@@ -57,8 +57,7 @@ public class DictionaryCacheLoaderImplTest {
     columnIdentifier = new ColumnIdentifier("1", columnProperties, DataTypes.STRING);
     dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType(), CarbonStorePath.getCarbonTablePath("/tmp",
-            carbonTableIdentifier));
+            columnIdentifier.getDataType());
     dictionaryCacheLoader = new DictionaryCacheLoaderImpl(dictionaryColumnUniqueIdentifier);
     dictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
     new MockUp<CarbonDictionaryReaderImpl>() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/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 c0b822b..89fbf2d 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
@@ -219,8 +219,7 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
 	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType(),
-            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
+            columnIdentifier.getDataType());
     Map<String, Integer> dataToSurrogateKeyMap = new HashMap<>(data.size());
     int surrogateKey = 0;
     List<Integer> invertedIndexList = new ArrayList<>(data.size());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/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 7450382..70f1740 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
@@ -53,8 +53,7 @@ public class CarbonDictionaryReaderImplTest {
             new CarbonTableIdentifier("dbName", "tableName", UUID.randomUUID().toString()));
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType(),
-        CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
+            columnIdentifier.getDataType());
     carbonDictionaryReaderImpl =
         new CarbonDictionaryReaderImpl(dictionaryColumnUniqueIdentifier);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/test/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java b/core/src/test/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
index b4c791c..68d34cf 100644
--- a/core/src/test/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
@@ -70,8 +70,7 @@ public class CarbonDictionarySortIndexReaderImplTest {
     ColumnIdentifier columnIdentifier = new ColumnIdentifier("Name", null, null);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType(),
-            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
+            columnIdentifier.getDataType());
     CarbonDictionaryWriter dictionaryWriter =
         new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
     String metaFolderPath =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/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 4fca00e..308d041 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
@@ -106,8 +106,7 @@ public class CarbonDictionaryWriterImplTest {
     absoluteTableIdentifier = AbsoluteTableIdentifier.from(carbonStorePath, carbonTableIdentifier);
     this.dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType(),
-            CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier));
+            columnIdentifier.getDataType());
     deleteStorePath();
     prepareDataSet();
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
index 2b5cc85..f74a254 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
@@ -63,8 +63,7 @@ public class CarbonDictionarySortIndexWriterImplTest {
     columnIdentifier = new ColumnIdentifier("Name", null, null);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType(),
-            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
+            columnIdentifier.getDataType());
     dictionaryWriter =
         new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
     dictionarySortIndexWriter =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
index bf25426..1191a38 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
@@ -79,8 +79,7 @@ public class CarbonRecordReader<T> extends AbstractRecordReader<T> {
     }
     List<TableBlockInfo> tableBlockInfoList = CarbonInputSplit.createBlocks(splitList);
     queryModel.setTableBlockInfos(tableBlockInfoList);
-    readSupport.initialize(queryModel.getProjectionColumns(),
-        queryModel.getAbsoluteTableIdentifier());
+    readSupport.initialize(queryModel.getProjectionColumns(), queryModel.getTable());
     try {
       carbonIterator = new ChunkRowIterator(queryExecutor.execute(queryModel));
     } catch (QueryExecutionException e) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java
index b535aea..c126e95 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java
@@ -18,7 +18,7 @@ package org.apache.carbondata.hadoop.readsupport;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 
 /**
@@ -30,10 +30,9 @@ public interface CarbonReadSupport<T> {
    * Initialization if needed based on the projected column list
    *
    * @param carbonColumns column list
-   * @param absoluteTableIdentifier table identifier
+   * @param carbonTable table identifier
    */
-  void initialize(CarbonColumn[] carbonColumns,
-      AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException;
+  void initialize(CarbonColumn[] carbonColumns, CarbonTable carbonTable) throws IOException;
 
   /**
    * convert column data back to row representation

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
index f4927dd..9851fe8 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
@@ -23,12 +23,12 @@ import org.apache.carbondata.core.cache.CacheProvider;
 import org.apache.carbondata.core.cache.CacheType;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
 
 /**
@@ -49,10 +49,10 @@ public class DictionaryDecodeReadSupport<T> implements CarbonReadSupport<T> {
    * for column dictionary involved in decoding.
    *
    * @param carbonColumns column list
-   * @param absoluteTableIdentifier table identifier
+   * @param carbonTable table identifier
    */
   @Override public void initialize(CarbonColumn[] carbonColumns,
-      AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException {
+      CarbonTable carbonTable) throws IOException {
     this.carbonColumns = carbonColumns;
     dictionaries = new Dictionary[carbonColumns.length];
     dataTypes = new DataType[carbonColumns.length];
@@ -63,10 +63,11 @@ public class DictionaryDecodeReadSupport<T> implements CarbonReadSupport<T> {
         Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache = cacheProvider
             .createCache(CacheType.FORWARD_DICTIONARY);
         dataTypes[i] = carbonColumns[i].getDataType();
+        String dictionaryPath = carbonTable.getTableInfo().getFactTable().getTableProperties()
+            .get(CarbonCommonConstants.DICTIONARY_PATH);
         dictionaries[i] = forwardDictionaryCache.get(new DictionaryColumnUniqueIdentifier(
-            absoluteTableIdentifier,
-            carbonColumns[i].getColumnIdentifier(), dataTypes[i],
-            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)));
+            carbonTable.getAbsoluteTableIdentifier(),
+            carbonColumns[i].getColumnIdentifier(), dataTypes[i], dictionaryPath));
       } else {
         dataTypes[i] = carbonColumns[i].getDataType();
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/RawDataReadSupport.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/RawDataReadSupport.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/RawDataReadSupport.java
index 7e7d414..b2cd450 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/RawDataReadSupport.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/RawDataReadSupport.java
@@ -16,7 +16,7 @@
  */
 package org.apache.carbondata.hadoop.readsupport.impl;
 
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
 
@@ -26,8 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
 public class RawDataReadSupport implements CarbonReadSupport<InternalRow> {
 
   @Override
-  public void initialize(CarbonColumn[] carbonColumns,
-      AbsoluteTableIdentifier absoluteTableIdentifier) { }
+  public void initialize(CarbonColumn[] carbonColumns, CarbonTable carbonTable) { }
 
   /**
    * return column data as InternalRow

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
index e6feb93..a6e9563 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
@@ -33,7 +34,6 @@ import org.apache.carbondata.core.scan.complextypes.PrimitiveQueryType;
 import org.apache.carbondata.core.scan.complextypes.StructQueryType;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;
@@ -95,10 +95,11 @@ public class CarbonStreamInputFormat extends FileInputFormat<Void, Object> {
       } else {
         boolean isDirectDictionary =
             CarbonUtil.hasEncoding(child.getEncoder(), Encoding.DIRECT_DICTIONARY);
+        String dictionaryPath = carbontable.getTableInfo().getFactTable().getTableProperties()
+            .get(CarbonCommonConstants.DICTIONARY_PATH);
         DictionaryColumnUniqueIdentifier dictionarIdentifier =
             new DictionaryColumnUniqueIdentifier(carbontable.getAbsoluteTableIdentifier(),
-                child.getColumnIdentifier(), child.getDataType(),
-                CarbonStorePath.getCarbonTablePath(carbontable.getAbsoluteTableIdentifier()));
+                child.getColumnIdentifier(), child.getDataType(), dictionaryPath);
 
         queryType =
             new PrimitiveQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/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 bea1d5e..ab22945 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
@@ -334,9 +334,7 @@ public class StoreCreator {
       ColumnIdentifier columnIdentifier = new ColumnIdentifier(dims.get(i).getColumnId(), null, null);
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
           new DictionaryColumnUniqueIdentifier(table.getAbsoluteTableIdentifier(), columnIdentifier,
-              columnIdentifier.getDataType(), CarbonStorePath
-              .getCarbonTablePath(table.getAbsoluteTableIdentifier().getTablePath(),
-                  table.getCarbonTableIdentifier()));
+              columnIdentifier.getDataType());
       CarbonDictionaryWriter writer =
           new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
       for (String value : set[i]) {
@@ -346,9 +344,7 @@ public class StoreCreator {
       writer.commit();
       Dictionary dict = (Dictionary) dictCache.get(
           new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
-        		  columnIdentifier, dims.get(i).getDataType(),CarbonStorePath
-              .getCarbonTablePath(table.getAbsoluteTableIdentifier().getTablePath(),
-                  table.getCarbonTableIdentifier())));
+        		  columnIdentifier, dims.get(i).getDataType()));
       CarbonDictionarySortInfoPreparator preparator =
           new CarbonDictionarySortInfoPreparator();
       List<String> newDistinctValues = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
----------------------------------------------------------------------
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
index 2f770cd..febca50 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonDictionaryDecodeReadSupport.java
@@ -28,10 +28,11 @@ import org.apache.carbondata.core.cache.CacheProvider;
 import org.apache.carbondata.core.cache.CacheType;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -71,10 +72,10 @@ public class CarbonDictionaryDecodeReadSupport<T> implements CarbonReadSupport<T
    * for column dictionary involved in decoding.
    *
    * @param carbonColumns           column list
-   * @param absoluteTableIdentifier table identifier
+   * @param carbonTable table identifier
    */
   @Override public void initialize(CarbonColumn[] carbonColumns,
-      AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException {
+      CarbonTable carbonTable) throws IOException {
     this.carbonColumns = carbonColumns;
     dictionaries = new Dictionary[carbonColumns.length];
     dataTypes = new DataType[carbonColumns.length];
@@ -85,9 +86,11 @@ public class CarbonDictionaryDecodeReadSupport<T> implements CarbonReadSupport<T
         Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache = cacheProvider
             .createCache(CacheType.FORWARD_DICTIONARY);
         dataTypes[i] = carbonColumns[i].getDataType();
+        String dictionaryPath = carbonTable.getTableInfo().getFactTable().getTableProperties()
+            .get(CarbonCommonConstants.DICTIONARY_PATH);
         dictionaries[i] = forwardDictionaryCache.get(
-            new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
-                carbonColumns[i].getColumnIdentifier()));
+            new DictionaryColumnUniqueIdentifier(carbonTable.getAbsoluteTableIdentifier(),
+                carbonColumns[i].getColumnIdentifier(), dataTypes[i], dictionaryPath));
       } else {
         dataTypes[i] = carbonColumns[i].getDataType();
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveRecordReader.java b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveRecordReader.java
index b8bd0e2..57bcca3 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveRecordReader.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonHiveRecordReader.java
@@ -80,7 +80,7 @@ class CarbonHiveRecordReader extends CarbonRecordReader<ArrayWritable>
     List<TableBlockInfo> tableBlockInfoList = CarbonHiveInputSplit.createBlocks(splitList);
     queryModel.setTableBlockInfos(tableBlockInfoList);
     readSupport
-        .initialize(queryModel.getProjectionColumns(), queryModel.getAbsoluteTableIdentifier());
+        .initialize(queryModel.getProjectionColumns(), queryModel.getTable());
     try {
       carbonIterator = new ChunkRowIterator(queryExecutor.execute(queryModel));
     } catch (QueryExecutionException e) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
index a9e2094..fb2e06e 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
@@ -76,7 +76,7 @@ public class CarbondataRecordSet implements RecordSet {
     try {
 
       readSupport
-          .initialize(queryModel.getProjectionColumns(), queryModel.getAbsoluteTableIdentifier());
+          .initialize(queryModel.getProjectionColumns(), queryModel.getTable());
       CarbonIterator iterator = queryExecutor.execute(queryModel);
       CarbonVectorizedRecordReader vectorReader =
           new CarbonVectorizedRecordReader(queryExecutor, queryModel,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala b/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
index c8e74a3..8e58f9a 100644
--- a/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
+++ b/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala
@@ -22,9 +22,10 @@ import io.airlift.slice.Slices._
 
 import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
 import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryChunksWrapper, DictionaryColumnUniqueIdentifier}
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn
 import org.apache.carbondata.core.util.{CarbonUtil, DataTypeUtil}
 import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport
@@ -45,8 +46,7 @@ class CarbonDictionaryDecodeReadSupport[T] extends CarbonReadSupport[T] {
    * @param absoluteTableIdentifier table identifier
    */
 
-  override def initialize(carbonColumns: Array[CarbonColumn],
-      absoluteTableIdentifier: AbsoluteTableIdentifier) {
+  override def initialize(carbonColumns: Array[CarbonColumn], carbonTable: CarbonTable) {
 
     dictionaries = new Array[Dictionary](carbonColumns.length)
     dataTypes = new Array[DataType](carbonColumns.length)
@@ -61,9 +61,11 @@ class CarbonDictionaryDecodeReadSupport[T] extends CarbonReadSupport[T] {
           cacheProvider
             .createCache(CacheType.FORWARD_DICTIONARY)
         dataTypes(index) = carbonColumn.getDataType
+        val dictionaryPath: String = carbonTable.getTableInfo.getFactTable.getTableProperties
+          .get(CarbonCommonConstants.DICTIONARY_PATH)
         dictionaries(index) = forwardDictionaryCache
-          .get(new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
-            carbonColumn.getColumnIdentifier))
+          .get(new DictionaryColumnUniqueIdentifier(carbonTable.getAbsoluteTableIdentifier,
+            carbonColumn.getColumnIdentifier, dataTypes(index), dictionaryPath))
         dictionarySliceArray(index) = createSliceArrayBlock(dictionaries(index))
 
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/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 2e5310d..ed89be0 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
@@ -357,10 +357,7 @@ object CarbonDataStoreCreator {
         new DictionaryColumnUniqueIdentifier(
           table.getAbsoluteTableIdentifier,
           columnIdentifier,
-          columnIdentifier.getDataType,
-          CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier.getTablePath,
-            table.getCarbonTableIdentifier)
-        )
+          columnIdentifier.getDataType)
       val writer: CarbonDictionaryWriter = new CarbonDictionaryWriterImpl(
         dictionaryColumnUniqueIdentifier)
       for (value <- set(i)) {
@@ -373,10 +370,8 @@ object CarbonDataStoreCreator {
           new DictionaryColumnUniqueIdentifier(
             absoluteTableIdentifier,
             columnIdentifier,
-            dims.get(i).getDataType,
-            CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier.getTablePath,
-              table.getCarbonTableIdentifier)
-          ))
+            dims.get(i).getDataType)
+          )
         .asInstanceOf[Dictionary]
       val preparator: CarbonDictionarySortInfoPreparator =
         new CarbonDictionarySortInfoPreparator()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/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 44fc7ad..db29532 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
@@ -347,8 +347,7 @@ class CarbonGlobalDictionaryGenerateRDD(
           DictionaryColumnUniqueIdentifier(
         model.table,
         model.columnIdentifier(split.index),
-        model.columnIdentifier(split.index).getDataType,
-        CarbonStorePath.getCarbonTablePath(model.table))
+        model.columnIdentifier(split.index).getDataType)
       val pathService: PathService = CarbonCommonFactory.getPathService
       val carbonTablePath: CarbonTablePath =
         pathService

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index ccbc9f5..ee2c843 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -175,8 +175,7 @@ object GlobalDictionaryUtil {
         DictionaryColumnUniqueIdentifier(
       model.table,
       model.columnIdentifier(columnIndex),
-      model.columnIdentifier(columnIndex).getDataType,
-      CarbonStorePath.getCarbonTablePath(model.table))
+      model.columnIdentifier(columnIndex).getDataType)
     val writer: CarbonDictionaryWriter = dictService
       .getDictionaryWriter(dictionaryColumnUniqueIdentifier)
     try {
@@ -215,8 +214,7 @@ object GlobalDictionaryUtil {
           DictionaryColumnUniqueIdentifier(
             model.table,
             model.columnIdentifier(i),
-            model.columnIdentifier(i).getDataType,
-            CarbonStorePath.getCarbonTablePath(model.table))
+            model.columnIdentifier(i).getDataType)
       val set = new HashSet[String]
       if (model.dictFileExists(i)) {
         val reader: CarbonDictionaryReader = dictService.getDictionaryReader(
@@ -854,8 +852,7 @@ object GlobalDictionaryUtil {
           DictionaryColumnUniqueIdentifier(
             absoluteTableIdentifier,
             columnIdentifier,
-            columnIdentifier.getDataType,
-            carbonTablePath)
+            columnIdentifier.getDataType)
       val parsedValue = DataTypeUtil.normalizeColumnValueForItsDataType(defaultValue, columnSchema)
       val valuesBuffer = new mutable.HashSet[String]
       if (null != parsedValue) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/integration/spark2/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
index a4bc636..8402766 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
@@ -18,7 +18,7 @@ package org.apache.carbondata.spark.readsupport;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.hadoop.readsupport.impl.DictionaryDecodeReadSupport;
 
@@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
 public class SparkRowReadSupportImpl extends DictionaryDecodeReadSupport<InternalRow> {
 
   @Override public void initialize(CarbonColumn[] carbonColumns,
-      AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException {
+      CarbonTable carbonTable) throws IOException {
   }
 
   @Override public InternalRow readRow(Object[] data) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index f9dfd28..21992dd 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -34,13 +34,14 @@ import org.apache.spark.sql.types._
 
 import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
 import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.ColumnIdentifier
 import org.apache.carbondata.core.metadata.datatype.{DataTypes => CarbonDataTypes}
 import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
 import org.apache.carbondata.core.scan.executor.util.QueryUtil
 import org.apache.carbondata.core.util.DataTypeUtil
-import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 import org.apache.carbondata.spark.rdd.CarbonRDDWithTableInfo
 
@@ -69,9 +70,9 @@ case class CarbonDictionaryDecoder(
 
   override def doExecute(): RDD[InternalRow] = {
     attachTree(this, "execute") {
-      val absoluteTableIdentifiers = relations.map { relation =>
+      val tableNameToCarbonTableMapping = relations.map { relation =>
         val carbonTable = relation.carbonRelation.carbonRelation.metaData.carbonTable
-        (carbonTable.getTableName, carbonTable.getAbsoluteTableIdentifier)
+        (carbonTable.getTableName, carbonTable)
       }.toMap
 
       if (CarbonDictionaryDecoder.isRequiredToDecode(getDictionaryColumnIds)) {
@@ -80,7 +81,7 @@ case class CarbonDictionaryDecoder(
           val cacheProvider: CacheProvider = CacheProvider.getInstance
           val forwardDictionaryCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
             cacheProvider.createCache(CacheType.FORWARD_DICTIONARY)
-          val dicts: Seq[Dictionary] = getDictionary(absoluteTableIdentifiers,
+          val dicts: Seq[Dictionary] = getDictionary(tableNameToCarbonTableMapping,
             forwardDictionaryCache)
           val dictIndex = dicts.zipWithIndex.filter(x => x._1 != null).map(x => x._2)
           // add a task completion listener to clear dictionary that is a decisive factor for
@@ -122,16 +123,16 @@ case class CarbonDictionaryDecoder(
 
   override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
 
-    val absoluteTableIdentifiers = relations.map { relation =>
+    val tableNameToCarbonTableMapping = relations.map { relation =>
       val carbonTable = relation.carbonRelation.carbonRelation.metaData.carbonTable
-      (carbonTable.getTableName, carbonTable.getAbsoluteTableIdentifier)
+      (carbonTable.getTableName, carbonTable)
     }.toMap
 
     if (CarbonDictionaryDecoder.isRequiredToDecode(getDictionaryColumnIds)) {
       val cacheProvider: CacheProvider = CacheProvider.getInstance
       val forwardDictionaryCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
         cacheProvider.createCache(CacheType.FORWARD_DICTIONARY)
-      val dicts: Seq[ForwardDictionaryWrapper] = getDictionaryWrapper(absoluteTableIdentifiers,
+      val dicts: Seq[ForwardDictionaryWrapper] = getDictionaryWrapper(tableNameToCarbonTableMapping,
         forwardDictionaryCache)
 
       val exprs = child.output.map { exp =>
@@ -243,15 +244,16 @@ case class CarbonDictionaryDecoder(
     child.asInstanceOf[CodegenSupport].produce(ctx, this)
   }
 
-  private def getDictionary(atiMap: Map[String, AbsoluteTableIdentifier],
+  private def getDictionary(atiMap: Map[String, CarbonTable],
       cache: Cache[DictionaryColumnUniqueIdentifier, Dictionary]) = {
     val dicts: Seq[Dictionary] = getDictionaryColumnIds.map { f =>
       if (f._2 != null) {
         try {
+          val dictionaryPath = atiMap(f._1).getTableInfo.getFactTable.getTableProperties
+            .get(CarbonCommonConstants.DICTIONARY_PATH)
           cache.get(new DictionaryColumnUniqueIdentifier(
-            atiMap(f._1),
-            f._2, f._3.getDataType,
-            CarbonStorePath.getCarbonTablePath(atiMap(f._1))))
+            atiMap(f._1).getAbsoluteTableIdentifier,
+            f._2, f._3.getDataType, dictionaryPath))
         } catch {
           case _: Throwable => null
         }
@@ -262,7 +264,7 @@ case class CarbonDictionaryDecoder(
     dicts
   }
 
-  private def getDictionaryWrapper(atiMap: Map[String, AbsoluteTableIdentifier],
+  private def getDictionaryWrapper(atiMap: Map[String, CarbonTable],
       cache: Cache[DictionaryColumnUniqueIdentifier, Dictionary]) = {
     val allDictIdentifiers = new ArrayBuffer[DictionaryColumnUniqueIdentifier]()
     val dicts: Seq[ForwardDictionaryWrapper] = getDictionaryColumnIds.map {
@@ -273,19 +275,22 @@ case class CarbonDictionaryDecoder(
               if (null != carbonDimension.getColumnSchema.getParentColumnTableRelations &&
                   !carbonDimension
                     .getColumnSchema.getParentColumnTableRelations.isEmpty) {
-                (QueryUtil.getTableIdentifierForColumn(carbonDimension, atiMap(tableName)),
+                (QueryUtil
+                  .getTableIdentifierForColumn(carbonDimension,
+                    atiMap(tableName).getAbsoluteTableIdentifier),
                   new ColumnIdentifier(carbonDimension.getColumnSchema
                     .getParentColumnTableRelations.get(0).getColumnId,
                     carbonDimension.getColumnProperties,
                     carbonDimension.getDataType))
               } else {
-                (atiMap(tableName), columnIdentifier)
+                (atiMap(tableName).getAbsoluteTableIdentifier, columnIdentifier)
               }
+            val dictionaryPath = atiMap(tableName).getTableInfo.getFactTable.getTableProperties
+              .get(CarbonCommonConstants.DICTIONARY_PATH)
             val dictionaryColumnUniqueIdentifier = new DictionaryColumnUniqueIdentifier(
               newAbsoluteTableIdentifier,
               newColumnIdentifier, carbonDimension.getDataType,
-              CarbonStorePath
-                .getCarbonTablePath(newAbsoluteTableIdentifier))
+              dictionaryPath)
             allDictIdentifiers += dictionaryColumnUniqueIdentifier
             new ForwardDictionaryWrapper(dictionaryColumnUniqueIdentifier)
           } catch {
@@ -504,14 +509,14 @@ class CarbonDecoderRDD(
 
   override def internalCompute(split: Partition, context: TaskContext): Iterator[InternalRow] = {
     val tableInfo = getTableInfo
-    val absoluteTableIdentifiers = relations.map { _ =>
-      (tableInfo.getFactTable.getTableName, tableInfo.getOrCreateAbsoluteTableIdentifier())
+    val tableNameToCarbonTableMapping = relations.map { _ =>
+      (tableInfo.getFactTable.getTableName, CarbonTable.buildFromTableInfo(tableInfo))
     }.toMap
 
     val cacheProvider: CacheProvider = CacheProvider.getInstance
     val forwardDictionaryCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
       cacheProvider.createCache(CacheType.FORWARD_DICTIONARY)
-    val dicts: Seq[Dictionary] = getDictionary(absoluteTableIdentifiers,
+    val dicts: Seq[Dictionary] = getDictionary(tableNameToCarbonTableMapping,
       forwardDictionaryCache)
     val dictIndex = dicts.zipWithIndex.filter(x => x._1 != null).map(x => x._2)
     // add a task completion listener to clear dictionary that is a decisive factor for
@@ -547,15 +552,16 @@ class CarbonDecoderRDD(
     }
   }
 
-  private def getDictionary(atiMap: Map[String, AbsoluteTableIdentifier],
+  private def getDictionary(atiMap: Map[String, CarbonTable],
       cache: Cache[DictionaryColumnUniqueIdentifier, Dictionary]) = {
     val dicts: Seq[Dictionary] = getDictionaryColumnIds.map { f =>
       if (f._2 != null) {
         try {
+          val dictionaryPath = atiMap(f._1).getTableInfo.getFactTable.getTableProperties
+            .get(CarbonCommonConstants.DICTIONARY_PATH)
           cache.get(new DictionaryColumnUniqueIdentifier(
-            atiMap(f._1),
-            f._2, f._3.getDataType,
-            CarbonStorePath.getCarbonTablePath(atiMap(f._1))))
+            atiMap(f._1).getAbsoluteTableIdentifier,
+            f._2, f._3.getDataType, dictionaryPath))
         } catch {
           case _: Throwable => null
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
index 8467f8d..f9519f8 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
@@ -43,9 +43,7 @@ object DictionaryTestCaseUtil {
     val tableIdentifier = new CarbonTableIdentifier(table.getDatabaseName, table.getTableName, "uniqueid")
     val  absoluteTableIdentifier = AbsoluteTableIdentifier.from(table.getTablePath, tableIdentifier)
     val columnIdentifier = new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
-      dimension.getColumnIdentifier, dimension.getDataType,
-      CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier)
-    )
+      dimension.getColumnIdentifier, dimension.getDataType)
     val dict = CarbonLoaderUtil.getDictionary(columnIdentifier)
     assert(dict.getSurrogateKey(value) != CarbonCommonConstants.INVALID_SURROGATE_KEY)
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/29dc3028/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
index f70e38e..55fa23b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
@@ -43,7 +43,6 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.processing.loading.dictionary.DictionaryServerClientDictionary;
 import org.apache.carbondata.processing.loading.dictionary.DirectDictionary;
 import org.apache.carbondata.processing.loading.dictionary.PreCreatedDictionary;
@@ -130,8 +129,7 @@ public class PrimitiveDataType implements GenericDataType<Object> {
     this.carbonDimension = carbonDimension;
     DictionaryColumnUniqueIdentifier identifier =
         new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
-            carbonDimension.getColumnIdentifier(), carbonDimension.getDataType(),
-            CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier));
+            carbonDimension.getColumnIdentifier(), carbonDimension.getDataType());
     try {
       if (carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
         dictionaryGenerator = new DirectDictionary(DirectDictionaryKeyGeneratorFactory